From f990947e9c6ca10cf860c5bf13bef92bc802d71a Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 6 Nov 2023 13:29:44 +0300 Subject: [PATCH 01/44] Discover ordering of complex expressions in group by and window partition by --- .../enforce_distribution.rs | 18 +- .../src/physical_optimizer/enforce_sorting.rs | 313 +++++++++++++++++ .../replace_with_order_preserving_variants.rs | 61 ++-- datafusion/physical-expr/src/equivalence.rs | 319 +++++++++++++----- datafusion/physical-plan/src/projection.rs | 49 +-- .../sqllogictest/test_files/groupby.slt | 45 +++ 6 files changed, 643 insertions(+), 162 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index ee6e11bd271a..27e2831c0371 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -3789,23 +3789,27 @@ mod tests { fn repartition_transitively_past_sort_with_projection_and_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, projection_exec_with_alias( filter_exec(parquet_exec()), - vec![("a".to_string(), "a".to_string())], + vec![ + ("a".to_string(), "a".to_string()), + ("b".to_string(), "b".to_string()), + ("c".to_string(), "c".to_string()), + ], ), false, ); let expected = &[ - "SortPreservingMergeExec: [c@2 ASC]", + "SortPreservingMergeExec: [a@0 ASC]", // Expect repartition on the input to the sort (as it can benefit from additional parallelism) - "SortExec: expr=[c@2 ASC]", - "ProjectionExec: expr=[a@0 as a]", + "SortExec: expr=[a@0 ASC]", + "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", "FilterExec: c@2 = 0", // repartition is lowest down "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", @@ -3815,9 +3819,9 @@ mod tests { assert_optimized!(expected, plan.clone(), true); let expected_first_sort_enforcement = &[ - "SortExec: expr=[c@2 ASC]", + "SortExec: expr=[a@0 ASC]", "CoalescePartitionsExec", - "ProjectionExec: expr=[a@0 as a]", + "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 4779ced44f1a..2d4aee583a7b 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -2278,3 +2278,316 @@ mod tests { Ok(()) } } + +// #[cfg(test)] +// 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(2); +// let ctx = SessionContext::new_with_config(config); +// +// ctx.sql( +// "CREATE unbounded EXTERNAL TABLE csv_with_timestamps ( +// name VARCHAR, +// ts TIMESTAMP +// ) +// STORED AS CSV +// WITH ORDER (ts DESC) +// LOCATION '../core/tests/data/timestamps.csv'", +// ) +// .await?; +// +// let sql = "SELECT date_bin('15 minutes', ts) +// FROM csv_with_timestamps +// GROUP BY (date_bin('15 minutes', ts)) +// 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 batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; +// print_batches(&batches)?; +// +// let expected = vec![ +// "GlobalLimitExec: skip=0, fetch=5", +// " CoalescePartitionsExec", +// " AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8(\"15 minutes\"),csv_with_timestamps.ts)@0 as date_bin(Utf8(\"15 minutes\"),csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted", +// " CoalesceBatchesExec: target_batch_size=8192", +// " SortPreservingRepartitionExec: partitioning=Hash([date_bin(Utf8(\"15 minutes\"),csv_with_timestamps.ts)@0], 2), input_partitions=2, sort_exprs=date_bin(Utf8(\"15 minutes\"),csv_with_timestamps.ts)@0 DESC", +// " AggregateExec: mode=Partial, gby=[date_bin(900000000000, ts@0) as date_bin(Utf8(\"15 minutes\"),csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted", +// " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", +// " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], infinite_source=true, output_ordering=[ts@0 DESC], has_header=false", +// ]; +// // 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 expected = [ +// "+------------+", +// "| nth_value1 |", +// "+------------+", +// "| 2 |", +// "| 2 |", +// "| 2 |", +// "| 2 |", +// "| 2 |", +// "+------------+", +// ]; +// assert_batches_eq!(expected, &batches); +// Ok(()) +// } +// +// #[tokio::test] +// async fn test_query3() -> Result<()> { +// let config = SessionConfig::new().with_target_partitions(1); +// let ctx = SessionContext::new_with_config(config); +// +// ctx.sql( +// "CREATE EXTERNAL TABLE csv_with_timestamps ( +// name VARCHAR, +// ts TIMESTAMP +// ) +// STORED AS CSV +// WITH ORDER (ts DESC) +// LOCATION '../core/tests/data/timestamps.csv'", +// ) +// .await?; +// +// let sql = " SELECT datebin FROM( +// SELECT date_bin('15 minutes', ts) as datebin +// FROM csv_with_timestamps) +// GROUP BY datebin; +// "; +// +// 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 batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; +// print_batches(&batches)?; +// +// let expected = vec![ +// "ProjectionExec: expr=[NTH_VALUE(annotated_data_finite2.d,Int64(2)) ORDER BY [annotated_data_finite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as nth_value1]", +// " GlobalLimitExec: skip=0, fetch=5", +// " BoundedWindowAggExec: wdw=[NTH_VALUE(annotated_data_finite2.d,Int64(2)) ORDER BY [annotated_data_finite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: \"NTH_VALUE(annotated_data_finite2.d,Int64(2)) ORDER BY [annotated_data_finite2.a 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, 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 expected = [ +// "+------------+", +// "| nth_value1 |", +// "+------------+", +// "| 2 |", +// "| 2 |", +// "| 2 |", +// "| 2 |", +// "| 2 |", +// "+------------+", +// ]; +// assert_batches_eq!(expected, &batches); +// Ok(()) +// } +// +// #[tokio::test] +// async fn test_query2() -> Result<()> { +// let config = SessionConfig::new().with_target_partitions(1); +// let ctx = SessionContext::new_with_config(config); +// +// ctx.sql( +// "CREATE EXTERNAL TABLE csv_with_timestamps ( +// name VARCHAR, +// ts TIMESTAMP +// ) +// STORED AS CSV +// WITH ORDER (ts DESC) +// LOCATION '../core/tests/data/timestamps.csv'", +// ) +// .await?; +// +// let sql = "SELECT ts + INTERVAL '15 minutes' +// FROM csv_with_timestamps +// GROUP BY (ts + INTERVAL '15 minutes') +// 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 batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; +// print_batches(&batches)?; +// +// let expected = vec![ +// "ProjectionExec: expr=[NTH_VALUE(annotated_data_finite2.d,Int64(2)) ORDER BY [annotated_data_finite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as nth_value1]", +// " GlobalLimitExec: skip=0, fetch=5", +// " BoundedWindowAggExec: wdw=[NTH_VALUE(annotated_data_finite2.d,Int64(2)) ORDER BY [annotated_data_finite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: \"NTH_VALUE(annotated_data_finite2.d,Int64(2)) ORDER BY [annotated_data_finite2.a 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, 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 expected = [ +// "+------------+", +// "| nth_value1 |", +// "+------------+", +// "| 2 |", +// "| 2 |", +// "| 2 |", +// "| 2 |", +// "| 2 |", +// "+------------+", +// ]; +// assert_batches_eq!(expected, &batches); +// Ok(()) +// } +// +// #[tokio::test] +// async fn test_query4() -> Result<()> { +// let config = SessionConfig::new().with_target_partitions(1); +// let ctx = SessionContext::new_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 batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; +// print_batches(&batches)?; +// +// let expected = vec![ +// "ProjectionExec: expr=[NTH_VALUE(annotated_data_finite2.d,Int64(2)) ORDER BY [annotated_data_finite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as nth_value1]", +// " GlobalLimitExec: skip=0, fetch=5", +// " BoundedWindowAggExec: wdw=[NTH_VALUE(annotated_data_finite2.d,Int64(2)) ORDER BY [annotated_data_finite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: \"NTH_VALUE(annotated_data_finite2.d,Int64(2)) ORDER BY [annotated_data_finite2.a 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, 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 expected = [ +// "+------------+", +// "| nth_value1 |", +// "+------------+", +// "| 2 |", +// "| 2 |", +// "| 2 |", +// "| 2 |", +// "| 2 |", +// "+------------+", +// ]; +// assert_batches_eq!(expected, &batches); +// Ok(()) +// } +// +// #[tokio::test] +// async fn test_query5() -> Result<()> { +// let config = SessionConfig::new().with_target_partitions(1); +// let ctx = SessionContext::new_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 batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; +// print_batches(&batches)?; +// +// let expected = 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, actual, +// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" +// ); +// +// let expected = [ +// "+------------+", +// "| nth_value1 |", +// "+------------+", +// "| 2 |", +// "| 2 |", +// "| 2 |", +// "| 2 |", +// "| 2 |", +// "+------------+", +// ]; +// assert_batches_eq!(expected, &batches); +// 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 0c2f21d11acd..cb7067263e1b 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 @@ -360,11 +360,11 @@ mod tests { " SortExec: expr=[a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) } @@ -400,7 +400,7 @@ mod tests { " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC], has_header=true", ]; let expected_optimized = [ @@ -411,7 +411,7 @@ mod tests { " SortPreservingMergeExec: [a@0 ASC]", " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC], has_header=true", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) @@ -435,13 +435,13 @@ mod tests { " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", " FilterExec: c@2 > 3", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " FilterExec: c@2 > 3", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); @@ -468,13 +468,13 @@ mod tests { " FilterExec: c@2 > 3", " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@2 > 3", " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) } @@ -502,14 +502,14 @@ mod tests { " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@2 > 3", " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) } @@ -532,13 +532,13 @@ mod tests { " FilterExec: c@2 > 3", " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; let expected_optimized = ["CoalescePartitionsExec", " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@2 > 3", " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) } @@ -566,7 +566,7 @@ mod tests { " FilterExec: c@2 > 3", " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true" + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true" ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", @@ -575,7 +575,7 @@ mod tests { " FilterExec: c@2 > 3", " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) @@ -601,12 +601,12 @@ mod tests { " SortExec: expr=[c@2 ASC]", " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; let expected_optimized = ["SortPreservingMergeExec: [c@2 ASC]", " SortExec: expr=[c@2 ASC]", " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) } @@ -626,11 +626,11 @@ mod tests { " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) } @@ -666,7 +666,7 @@ mod tests { " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; let expected_optimized = [ @@ -678,7 +678,7 @@ mod tests { " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) @@ -712,29 +712,29 @@ mod tests { let expected_input = [ "SortPreservingMergeExec: [a@0 ASC]", " SortExec: expr=[a@0 ASC]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c@2)]", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC]", " SortExec: expr=[a@0 ASC]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c@2)]", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) @@ -755,11 +755,11 @@ mod tests { " SortExec: expr=[a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) } @@ -859,7 +859,7 @@ mod tests { HashJoinExec::try_new( left, right, - vec![(Column::new("c", 1), Column::new("c", 1))], + vec![(Column::new("c", 2), Column::new("c", 2))], None, &JoinType::Inner, PartitionMode::Partitioned, @@ -887,7 +887,6 @@ mod tests { infinite_source: bool, ) -> Arc { let sort_exprs = sort_exprs.into_iter().collect(); - let projection: Vec = vec![0, 2, 3]; Arc::new(CsvExec::new( FileScanConfig { @@ -898,7 +897,7 @@ mod tests { 100, )]], statistics: Statistics::new_unknown(schema), - projection: Some(projection), + projection: None, limit: None, table_partition_cols: vec![], output_ordering: vec![sort_exprs], diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index d8aa09b90460..29839724071c 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 std::collections::HashSet; +use std::collections::{HashMap, HashSet}; use std::hash::Hash; use std::sync::Arc; @@ -28,12 +28,13 @@ use crate::{ }; use arrow::datatypes::SchemaRef; -use arrow_schema::SortOptions; +use arrow_schema::{Field, Schema, SortOptions}; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{JoinSide, JoinType, Result}; use indexmap::map::Entry; use indexmap::IndexMap; +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 @@ -48,6 +49,7 @@ pub struct ProjectionMapping { /// `(source expression)` --> `(target expression)` /// Indices in the vector corresponds to the indices after projection. inner: Vec<(Arc, Arc)>, + output_schema: SchemaRef, } impl ProjectionMapping { @@ -91,7 +93,33 @@ impl ProjectionMapping { inner.push((source_expr, target_expr)); } - Ok(Self { inner }) + + // Calculate output schema + let fields: Result> = expr + .iter() + .map(|(e, name)| { + let mut field = Field::new( + name, + e.data_type(input_schema)?, + e.nullable(input_schema)?, + ); + field.set_metadata( + get_field_metadata(e, input_schema).unwrap_or_default(), + ); + + Ok(field) + }) + .collect(); + + let output_schema = Arc::new(Schema::new_with_metadata( + fields?, + input_schema.metadata().clone(), + )); + + Ok(Self { + inner, + output_schema, + }) } /// Iterate over pairs of (source, target) expressions @@ -100,6 +128,58 @@ impl ProjectionMapping { ) -> impl Iterator, Arc)> + '_ { self.inner.iter() } + + /// Get a reference to the output schema after projection applied + pub fn output_schema(&self) -> SchemaRef { + self.output_schema.clone() + } + + /// Return target expressions for each key. + pub fn target_exprs( + &self, + exprs: &[Arc], + ) -> Option>> { + exprs + .iter() + .map(|expr| self.target_expr(expr)) + .collect::>>() + } + + /// Return target expression for given expr + /// None means given key is not found. + pub fn target_expr( + &self, + expr: &Arc, + ) -> Option> { + if let Some(idx) = self + .inner + .iter() + .position(|(source, _target)| source.eq(expr)) + { + let (_source, target) = &self.inner[idx]; + Some(target.clone()) + } else { + None + } + } +} + +/// If e is a direct column reference, returns the field level +/// metadata for that field, if any. Otherwise returns None +fn get_field_metadata( + e: &Arc, + input_schema: &Schema, +) -> Option> { + let name = if let Some(column) = e.as_any().downcast_ref::() { + column.name() + } else { + return None; + }; + + input_schema + .field_with_name(name) + .ok() + .map(|f| f.metadata().clone()) } /// An `EquivalenceGroup` is a collection of `EquivalenceClass`es where each @@ -320,24 +400,31 @@ impl EquivalenceGroup { mapping: &ProjectionMapping, expr: &Arc, ) -> Option> { - let children = expr.children(); - if children.is_empty() { - 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_equivalence_class(source) - .map_or(false, |group| physical_exprs_contains(group, expr)) - { - return Some(target.clone()); - } + for (source, target) in mapping.iter() { + // If we match the source, 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) { + return Some(target.clone()); + } + } + for (source, target) in mapping.iter() { + // If we match 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 self + .get_equivalence_class(source) + .map_or(false, |group| physical_exprs_contains(group, expr)) + { + return Some(target.clone()); } } // Project a non-leaf expression by projecting its children. - else if let Some(children) = children + let children = expr.children(); + if children.is_empty() { + return None; + } else if let Some(children) = children .into_iter() .map(|child| self.project_expr(mapping, &child)) .collect::>>() @@ -557,7 +644,7 @@ impl OrderingEquivalenceClass { fn remove_redundant_entries(&mut self) { let mut idx = 0; while idx < self.orderings.len() { - let mut removal = false; + let mut removal = self.orderings[idx].is_empty(); for (ordering_idx, ordering) in self.orderings[0..idx].iter().enumerate() { if let Some(right_finer) = finer_side(ordering, &self.orderings[idx]) { if right_finer { @@ -872,6 +959,51 @@ impl EquivalenceProperties { /// Checks whether the given sort requirements are satisfied by any of the /// existing orderings. pub fn ordering_satisfy_requirement(&self, reqs: LexRequirementRef) -> bool { + let mut proj_exprs = self + .schema + .fields + .iter() + .enumerate() + .map(|(idx, field)| { + let col = + Arc::new(Column::new(field.name(), idx)) as Arc; + (col, field.name().to_string()) + }) + .collect::>(); + let exprs_used = reqs + .iter() + .map(|sort_req| sort_req.expr.clone()) + .collect::>(); + let complex_proj_exprs = exprs_used + .iter() + .flat_map(|expr| { + if expr.as_any().is::() { + // Do not project column expressions + None + } else { + Some((expr.clone(), expr.to_string())) + } + }) + .collect::>(); + proj_exprs.extend(complex_proj_exprs); + let projection_mapping = + ProjectionMapping::try_new(&proj_exprs, self.schema()).unwrap(); + let projected_eqs = + self.project(&projection_mapping, projection_mapping.output_schema()); + if let Some(target_exprs) = projection_mapping.target_exprs(&exprs_used) { + let projected_reqs = izip!(target_exprs.into_iter(), reqs.iter()) + .map(|(target_expr, sort_req)| PhysicalSortRequirement { + expr: target_expr, + options: sort_req.options, + }) + .collect::>(); + projected_eqs.ordering_satisfy_requirement_helper(&projected_reqs) + } else { + false + } + } + + fn ordering_satisfy_requirement_helper(&self, reqs: LexRequirementRef) -> bool { // First, standardize the given requirement: let normalized_reqs = self.normalize_sort_requirements(reqs); if normalized_reqs.is_empty() { @@ -1049,10 +1181,15 @@ impl EquivalenceProperties { }]); } } + let target_constants = self + .constants + .iter() + .flat_map(|constant| projection_mapping.target_expr(constant)) + .collect::>(); Self { eq_group: self.eq_group.project(projection_mapping), oeq_class: OrderingEquivalenceClass::new(projected_orderings), - constants: vec![], + constants: target_constants, schema: output_schema, } } @@ -1070,6 +1207,42 @@ impl EquivalenceProperties { pub fn find_longest_permutation( &self, exprs: &[Arc], + ) -> (LexOrdering, Vec) { + let mut proj_exprs = self + .schema + .fields + .iter() + .enumerate() + .map(|(idx, field)| { + let col = + Arc::new(Column::new(field.name(), idx)) as Arc; + (col, field.name().to_string()) + }) + .collect::>(); + let complex_proj_exprs = exprs + .iter() + .flat_map(|expr| { + if expr.as_any().is::() { + // Do not project column expressions + None + } else { + Some((expr.clone(), expr.to_string())) + } + }) + .collect::>(); + proj_exprs.extend(complex_proj_exprs); + let projection_mapping = + ProjectionMapping::try_new(&proj_exprs, self.schema()).unwrap(); + let projected = + self.project(&projection_mapping, projection_mapping.output_schema()); + projected.find_longest_permutation_helper( + &projection_mapping.target_exprs(exprs).unwrap_or_default(), + ) + } + + fn find_longest_permutation_helper( + &self, + exprs: &[Arc], ) -> (LexOrdering, Vec) { let normalized_exprs = self.eq_group.normalize_exprs(exprs.to_vec()); // Use a map to associate expression indices with sort options: @@ -1482,27 +1655,21 @@ mod tests { let input_properties = EquivalenceProperties::new(input_schema.clone()); let col_a = col("a", &input_schema)?; - - let out_schema = Arc::new(Schema::new(vec![ - Field::new("a1", DataType::Int64, true), - Field::new("a2", DataType::Int64, true), - Field::new("a3", DataType::Int64, true), - Field::new("a4", DataType::Int64, true), - ])); + // a as a1, a as a2, a as a3, a as a3 + let proj_exprs = vec![ + (col_a.clone(), "a1".to_string()), + (col_a.clone(), "a2".to_string()), + (col_a.clone(), "a3".to_string()), + (col_a.clone(), "a4".to_string()), + ]; + let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &input_schema)?; + let out_schema = projection_mapping.output_schema(); // 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)?; let col_a4 = &col("a4", &out_schema)?; - let projection_mapping = ProjectionMapping { - inner: 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(&projection_mapping, out_schema); // At the output a1=a2=a3=a4 @@ -1519,6 +1686,10 @@ mod tests { #[test] fn test_ordering_satisfy() -> Result<()> { + let input_schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int64, true), + Field::new("b", DataType::Int64, true), + ])); let crude = vec![PhysicalSortExpr { expr: Arc::new(Column::new("a", 0)), options: SortOptions::default(), @@ -1534,13 +1705,12 @@ mod tests { }, ]; // finer ordering satisfies, crude ordering should return true - let empty_schema = &Arc::new(Schema::empty()); - let mut eq_properties_finer = EquivalenceProperties::new(empty_schema.clone()); + let mut eq_properties_finer = EquivalenceProperties::new(input_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 eq_properties_crude = EquivalenceProperties::new(empty_schema.clone()); + let mut eq_properties_crude = EquivalenceProperties::new(input_schema.clone()); eq_properties_crude.oeq_class.push(crude.clone()); assert!(!eq_properties_crude.ordering_satisfy(&finer)); Ok(()) @@ -1753,8 +1923,8 @@ mod tests { table_data_with_properties.clone(), )?; let err_msg = format!( - "Error in test case requirement:{:?}, expected: {:?}", - requirement, expected + "Error in test case requirement:{:?}, expected: {:?}, eq_properties.constants: {:?}, eq_properties.eq_group: {:?}, eq_properties.oeq_class: {:?}", + requirement, expected, eq_properties.constants, eq_properties.eq_group, eq_properties.oeq_class ); // Check whether ordering_satisfy API result and // experimental result matches. @@ -1998,6 +2168,13 @@ mod tests { ], ], ), + // ------- TEST CASE 5 --------- + // Empty ordering + ( + vec![vec![]], + // No ordering in the state (empty ordering is ignored). + vec![], + ), ]; for (orderings, expected) in test_cases { let orderings = convert_to_orderings(&orderings); @@ -2786,35 +2963,26 @@ mod tests { #[test] fn project_empty_output_ordering() -> Result<()> { - let schema = Schema::new(vec![ + let schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::Int32, true), Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), - ]); - let mut eq_properties = EquivalenceProperties::new(Arc::new(schema.clone())); + ])); + let mut eq_properties = EquivalenceProperties::new(schema.clone()); let ordering = vec![PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), + expr: col("b", &schema)?, options: SortOptions::default(), }]; eq_properties.add_new_orderings([ordering]); - let projection_mapping = ProjectionMapping { - inner: 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), - ])); + // b as b_new, a as a_new + let proj_exprs = vec![ + (col("b", &schema)?, "b_new".to_string()), + (col("a", &schema)?, "a_new".to_string()), + ]; + let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &schema)?; + let orderings = eq_properties - .project(&projection_mapping, projection_schema) + .project(&projection_mapping, projection_mapping.output_schema()) .oeq_class() .output_ordering() .unwrap_or_default(); @@ -2827,29 +2995,20 @@ mod tests { orderings ); - let schema = Schema::new(vec![ + let schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::Int32, true), Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), - ]); - let eq_properties = EquivalenceProperties::new(Arc::new(schema)); - let projection_mapping = ProjectionMapping { - inner: 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 = eq_properties.project(&projection_mapping, projection_schema); + let eq_properties = EquivalenceProperties::new(schema.clone()); + // c as c_new, b as b_new + let proj_exprs = vec![ + (col("c", &schema)?, "c_new".to_string()), + (col("b", &schema)?, "b_new".to_string()), + ]; + let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &schema)?; + let projected = eq_properties + .project(&projection_mapping, projection_mapping.output_schema()); // After projection there is no ordering. assert!(projected.oeq_class().output_ordering().is_none()); diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index c5d94b08e0e1..7c6794e91e48 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -21,7 +21,6 @@ //! projection expressions. `SELECT` without `FROM` will only evaluate expressions. use std::any::Any; -use std::collections::HashMap; use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; @@ -33,7 +32,7 @@ use crate::{ ColumnStatistics, DisplayFormatType, ExecutionPlan, Partitioning, PhysicalExpr, }; -use arrow::datatypes::{Field, Schema, SchemaRef}; +use arrow::datatypes::SchemaRef; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::stats::Precision; use datafusion_common::Result; @@ -71,37 +70,17 @@ impl ProjectionExec { ) -> Result { let input_schema = input.schema(); - let fields: Result> = expr - .iter() - .map(|(e, name)| { - let mut field = Field::new( - name, - e.data_type(&input_schema)?, - e.nullable(&input_schema)?, - ); - field.set_metadata( - get_field_metadata(e, &input_schema).unwrap_or_default(), - ); - - Ok(field) - }) - .collect(); - - let schema = Arc::new(Schema::new_with_metadata( - fields?, - input_schema.metadata().clone(), - )); - // construct a map from the input expressions to the output expression of the Projection let projection_mapping = ProjectionMapping::try_new(&expr, &input_schema)?; let input_eqs = input.equivalence_properties(); - let project_eqs = input_eqs.project(&projection_mapping, schema.clone()); + let project_eqs = + input_eqs.project(&projection_mapping, projection_mapping.output_schema()); let output_ordering = project_eqs.oeq_class().output_ordering(); Ok(Self { expr, - schema, + schema: projection_mapping.output_schema(), input, output_ordering, projection_mapping, @@ -251,24 +230,6 @@ impl ExecutionPlan for ProjectionExec { } } -/// If e is a direct column reference, returns the field level -/// metadata for that field, if any. Otherwise returns None -fn get_field_metadata( - e: &Arc, - input_schema: &Schema, -) -> Option> { - let name = if let Some(column) = e.as_any().downcast_ref::() { - column.name() - } else { - return None; - }; - - input_schema - .field_with_name(name) - .ok() - .map(|f| f.metadata().clone()) -} - fn stats_projection( mut stats: Statistics, exprs: impl Iterator>, @@ -368,7 +329,7 @@ mod tests { use crate::expressions; use crate::test; - use arrow_schema::DataType; + use arrow_schema::{DataType, Field, Schema}; use datafusion_common::ScalarValue; #[tokio::test] diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index cb0b0b7c76a5..8b8d095659ec 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -3846,3 +3846,48 @@ ProjectionExec: expr=[SUM(alias1)@1 as SUM(DISTINCT t1.x), MAX(alias1)@2 as MAX( ------------------AggregateExec: mode=Partial, gby=[y@1 as y, CAST(t1.x AS Float64)t1.x@0 as alias1], aggr=[] --------------------ProjectionExec: expr=[CAST(x@0 AS Float64) as CAST(t1.x AS Float64)t1.x, y@1 as y] ----------------------MemoryExec: partitions=1, partition_sizes=[1] + +# create an unbounded table that contains ordered timestamp. +statement ok +CREATE UNBOUNDED EXTERNAL TABLE csv_with_timestamps ( + name VARCHAR, + ts TIMESTAMP +) +STORED AS CSV +WITH ORDER (ts DESC) +LOCATION '../core/tests/data/timestamps.csv' + +# below query should work in streaming mode. +query TT +EXPLAIN SELECT date_bin('15 minutes', ts) as time_chunks + FROM csv_with_timestamps + GROUP BY (date_bin('15 minutes', ts)) + ORDER BY time_chunks DESC + LIMIT 5 +---- +logical_plan +Limit: skip=0, fetch=5 +--Sort: time_chunks DESC NULLS FIRST, fetch=5 +----Projection: date_bin(Utf8("15 minutes"),csv_with_timestamps.ts) AS time_chunks +------Aggregate: groupBy=[[date_bin(IntervalMonthDayNano("900000000000"), csv_with_timestamps.ts) AS date_bin(Utf8("15 minutes"),csv_with_timestamps.ts)]], aggr=[[]] +--------TableScan: csv_with_timestamps projection=[ts] +physical_plan +GlobalLimitExec: skip=0, fetch=5 +--SortPreservingMergeExec: [time_chunks@0 DESC], fetch=5 +----ProjectionExec: expr=[date_bin(Utf8("15 minutes"),csv_with_timestamps.ts)@0 as time_chunks] +------AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("15 minutes"),csv_with_timestamps.ts)@0 as date_bin(Utf8("15 minutes"),csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted +--------CoalesceBatchesExec: target_batch_size=2 +----------SortPreservingRepartitionExec: partitioning=Hash([date_bin(Utf8("15 minutes"),csv_with_timestamps.ts)@0], 8), input_partitions=8, sort_exprs=date_bin(Utf8("15 minutes"),csv_with_timestamps.ts)@0 DESC +------------AggregateExec: mode=Partial, gby=[date_bin(900000000000, ts@0) as date_bin(Utf8("15 minutes"),csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted +--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], infinite_source=true, output_ordering=[ts@0 DESC], has_header=false + +query P +SELECT date_bin('15 minutes', ts) as time_chunks + FROM csv_with_timestamps + GROUP BY (date_bin('15 minutes', ts)) + ORDER BY time_chunks DESC + LIMIT 5 +---- +2018-12-13T12:00:00 +2018-11-13T17:00:00 From ce982f0ae3cfa4e18f63d74fabcab9ff42bce757 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 6 Nov 2023 16:08:10 +0300 Subject: [PATCH 02/44] Remove unnecessary tests --- .../src/physical_optimizer/enforce_sorting.rs | 313 ------------------ 1 file changed, 313 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 2d4aee583a7b..4779ced44f1a 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -2278,316 +2278,3 @@ mod tests { Ok(()) } } - -// #[cfg(test)] -// 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(2); -// let ctx = SessionContext::new_with_config(config); -// -// ctx.sql( -// "CREATE unbounded EXTERNAL TABLE csv_with_timestamps ( -// name VARCHAR, -// ts TIMESTAMP -// ) -// STORED AS CSV -// WITH ORDER (ts DESC) -// LOCATION '../core/tests/data/timestamps.csv'", -// ) -// .await?; -// -// let sql = "SELECT date_bin('15 minutes', ts) -// FROM csv_with_timestamps -// GROUP BY (date_bin('15 minutes', ts)) -// 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 batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; -// print_batches(&batches)?; -// -// let expected = vec![ -// "GlobalLimitExec: skip=0, fetch=5", -// " CoalescePartitionsExec", -// " AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8(\"15 minutes\"),csv_with_timestamps.ts)@0 as date_bin(Utf8(\"15 minutes\"),csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted", -// " CoalesceBatchesExec: target_batch_size=8192", -// " SortPreservingRepartitionExec: partitioning=Hash([date_bin(Utf8(\"15 minutes\"),csv_with_timestamps.ts)@0], 2), input_partitions=2, sort_exprs=date_bin(Utf8(\"15 minutes\"),csv_with_timestamps.ts)@0 DESC", -// " AggregateExec: mode=Partial, gby=[date_bin(900000000000, ts@0) as date_bin(Utf8(\"15 minutes\"),csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted", -// " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", -// " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], infinite_source=true, output_ordering=[ts@0 DESC], has_header=false", -// ]; -// // 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 expected = [ -// "+------------+", -// "| nth_value1 |", -// "+------------+", -// "| 2 |", -// "| 2 |", -// "| 2 |", -// "| 2 |", -// "| 2 |", -// "+------------+", -// ]; -// assert_batches_eq!(expected, &batches); -// Ok(()) -// } -// -// #[tokio::test] -// async fn test_query3() -> Result<()> { -// let config = SessionConfig::new().with_target_partitions(1); -// let ctx = SessionContext::new_with_config(config); -// -// ctx.sql( -// "CREATE EXTERNAL TABLE csv_with_timestamps ( -// name VARCHAR, -// ts TIMESTAMP -// ) -// STORED AS CSV -// WITH ORDER (ts DESC) -// LOCATION '../core/tests/data/timestamps.csv'", -// ) -// .await?; -// -// let sql = " SELECT datebin FROM( -// SELECT date_bin('15 minutes', ts) as datebin -// FROM csv_with_timestamps) -// GROUP BY datebin; -// "; -// -// 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 batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; -// print_batches(&batches)?; -// -// let expected = vec![ -// "ProjectionExec: expr=[NTH_VALUE(annotated_data_finite2.d,Int64(2)) ORDER BY [annotated_data_finite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as nth_value1]", -// " GlobalLimitExec: skip=0, fetch=5", -// " BoundedWindowAggExec: wdw=[NTH_VALUE(annotated_data_finite2.d,Int64(2)) ORDER BY [annotated_data_finite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: \"NTH_VALUE(annotated_data_finite2.d,Int64(2)) ORDER BY [annotated_data_finite2.a 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, 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 expected = [ -// "+------------+", -// "| nth_value1 |", -// "+------------+", -// "| 2 |", -// "| 2 |", -// "| 2 |", -// "| 2 |", -// "| 2 |", -// "+------------+", -// ]; -// assert_batches_eq!(expected, &batches); -// Ok(()) -// } -// -// #[tokio::test] -// async fn test_query2() -> Result<()> { -// let config = SessionConfig::new().with_target_partitions(1); -// let ctx = SessionContext::new_with_config(config); -// -// ctx.sql( -// "CREATE EXTERNAL TABLE csv_with_timestamps ( -// name VARCHAR, -// ts TIMESTAMP -// ) -// STORED AS CSV -// WITH ORDER (ts DESC) -// LOCATION '../core/tests/data/timestamps.csv'", -// ) -// .await?; -// -// let sql = "SELECT ts + INTERVAL '15 minutes' -// FROM csv_with_timestamps -// GROUP BY (ts + INTERVAL '15 minutes') -// 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 batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; -// print_batches(&batches)?; -// -// let expected = vec![ -// "ProjectionExec: expr=[NTH_VALUE(annotated_data_finite2.d,Int64(2)) ORDER BY [annotated_data_finite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as nth_value1]", -// " GlobalLimitExec: skip=0, fetch=5", -// " BoundedWindowAggExec: wdw=[NTH_VALUE(annotated_data_finite2.d,Int64(2)) ORDER BY [annotated_data_finite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: \"NTH_VALUE(annotated_data_finite2.d,Int64(2)) ORDER BY [annotated_data_finite2.a 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, 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 expected = [ -// "+------------+", -// "| nth_value1 |", -// "+------------+", -// "| 2 |", -// "| 2 |", -// "| 2 |", -// "| 2 |", -// "| 2 |", -// "+------------+", -// ]; -// assert_batches_eq!(expected, &batches); -// Ok(()) -// } -// -// #[tokio::test] -// async fn test_query4() -> Result<()> { -// let config = SessionConfig::new().with_target_partitions(1); -// let ctx = SessionContext::new_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 batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; -// print_batches(&batches)?; -// -// let expected = vec![ -// "ProjectionExec: expr=[NTH_VALUE(annotated_data_finite2.d,Int64(2)) ORDER BY [annotated_data_finite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as nth_value1]", -// " GlobalLimitExec: skip=0, fetch=5", -// " BoundedWindowAggExec: wdw=[NTH_VALUE(annotated_data_finite2.d,Int64(2)) ORDER BY [annotated_data_finite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: \"NTH_VALUE(annotated_data_finite2.d,Int64(2)) ORDER BY [annotated_data_finite2.a 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, 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 expected = [ -// "+------------+", -// "| nth_value1 |", -// "+------------+", -// "| 2 |", -// "| 2 |", -// "| 2 |", -// "| 2 |", -// "| 2 |", -// "+------------+", -// ]; -// assert_batches_eq!(expected, &batches); -// Ok(()) -// } -// -// #[tokio::test] -// async fn test_query5() -> Result<()> { -// let config = SessionConfig::new().with_target_partitions(1); -// let ctx = SessionContext::new_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 batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; -// print_batches(&batches)?; -// -// let expected = 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, actual, -// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" -// ); -// -// let expected = [ -// "+------------+", -// "| nth_value1 |", -// "+------------+", -// "| 2 |", -// "| 2 |", -// "| 2 |", -// "| 2 |", -// "| 2 |", -// "+------------+", -// ]; -// assert_batches_eq!(expected, &batches); -// Ok(()) -// } -// } From 8dc081f5af1b12f484716c92dca682365e397dec Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 6 Nov 2023 16:34:28 +0300 Subject: [PATCH 03/44] Update comments --- datafusion/physical-expr/src/equivalence.rs | 160 ++++++++++++-------- 1 file changed, 94 insertions(+), 66 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 29839724071c..46979ad1795e 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -34,7 +34,6 @@ use datafusion_common::{JoinSide, JoinType, Result}; use indexmap::map::Entry; use indexmap::IndexMap; -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 @@ -134,7 +133,38 @@ impl ProjectionMapping { self.output_schema.clone() } - /// Return target expressions for each key. + /// This function projects ordering requirement according to projection. + /// + /// # Arguments + /// + /// * `lex_req` - Lexicographical ordering requirement. + /// + /// # Returns + /// + /// An `Option` containing the Lexicographical projected ordering requirement. + pub fn project_lex_reqs(&self, lex_req: LexRequirementRef) -> Option { + lex_req + .iter() + .map(|sort_req| { + self.target_expr(&sort_req.expr) + .map(|expr| PhysicalSortRequirement { + expr, + options: sort_req.options, + }) + }) + .collect::>>() + } + + /// This function returns target value for each expression + /// + /// # Arguments + /// + /// * `exprs` - Source (e.g key) physical expressions + /// + /// # Returns + /// + /// An `Option` containing a the targets (e.g value) for the source each expression. + /// Returns `Some(Vec<_>)` if all of the expressions are source in the projection mapping. pub fn target_exprs( &self, exprs: &[Arc], @@ -145,8 +175,16 @@ impl ProjectionMapping { .collect::>>() } - /// Return target expression for given expr - /// None means given key is not found. + /// This function returns target value for given expression + /// + /// # Arguments + /// + /// * `expr` - Source (e.g key) physical expression + /// + /// # Returns + /// + /// An `Option` containing a the target (e.g value) for the source expression. + /// `None` means that source is not found inside the mapping. pub fn target_expr( &self, expr: &Arc, @@ -402,8 +440,7 @@ impl EquivalenceGroup { ) -> Option> { for (source, target) in mapping.iter() { // If we match the source, 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. + // (a as a1, a + c) `a` projects to `a1`, and `binary_expr(a+b)` projects to `col(a+b)`. if source.eq(expr) { return Some(target.clone()); } @@ -411,8 +448,7 @@ impl EquivalenceGroup { for (source, target) in mapping.iter() { // If we match 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. + // (a as a1, a + c) and the equivalence class (a, b), expression `b` projects to `a1`. if self .get_equivalence_class(source) .map_or(false, |group| physical_exprs_contains(group, expr)) @@ -423,6 +459,7 @@ impl EquivalenceGroup { // Project a non-leaf expression by projecting its children. let children = expr.children(); if children.is_empty() { + // Leaf expression should be inside mapping. return None; } else if let Some(children) = children .into_iter() @@ -958,51 +995,25 @@ impl EquivalenceProperties { /// Checks whether the given sort requirements are satisfied by any of the /// existing orderings. + /// This function applies an implicit projection to itself before calling `ordering_satisfy_requirement_helper`. + /// This enables us to consider complex expressions during analysis. pub fn ordering_satisfy_requirement(&self, reqs: LexRequirementRef) -> bool { - let mut proj_exprs = self - .schema - .fields - .iter() - .enumerate() - .map(|(idx, field)| { - let col = - Arc::new(Column::new(field.name(), idx)) as Arc; - (col, field.name().to_string()) - }) - .collect::>(); - let exprs_used = reqs + let exprs = reqs .iter() .map(|sort_req| sort_req.expr.clone()) .collect::>(); - let complex_proj_exprs = exprs_used - .iter() - .flat_map(|expr| { - if expr.as_any().is::() { - // Do not project column expressions - None - } else { - Some((expr.clone(), expr.to_string())) - } - }) - .collect::>(); - proj_exprs.extend(complex_proj_exprs); - let projection_mapping = - ProjectionMapping::try_new(&proj_exprs, self.schema()).unwrap(); + let projection_mapping = self.implicit_projection_mapping(&exprs); let projected_eqs = self.project(&projection_mapping, projection_mapping.output_schema()); - if let Some(target_exprs) = projection_mapping.target_exprs(&exprs_used) { - let projected_reqs = izip!(target_exprs.into_iter(), reqs.iter()) - .map(|(target_expr, sort_req)| PhysicalSortRequirement { - expr: target_expr, - options: sort_req.options, - }) - .collect::>(); + if let Some(projected_reqs) = projection_mapping.project_lex_reqs(reqs) { projected_eqs.ordering_satisfy_requirement_helper(&projected_reqs) } else { false } } + /// Helper function to check whether the given sort requirements are satisfied by any of the + /// existing orderings. fn ordering_satisfy_requirement_helper(&self, reqs: LexRequirementRef) -> bool { // First, standardize the given requirement: let normalized_reqs = self.normalize_sort_requirements(reqs); @@ -1136,6 +1147,43 @@ impl EquivalenceProperties { (!meet.is_empty()).then_some(meet) } + /// Creates a projection mapping to support complex expressions. + /// All of the existing fields + complex expressions + /// (expressions that needs to be evaluated using existing fields) such (a+b), (date_bin(ts), etc.) + /// are projected. + /// With this API, we can determine ordering properties of complex expressions without actually evaluating them. + fn implicit_projection_mapping( + &self, + exprs: &[Arc], + ) -> ProjectionMapping { + // Project existing fields as is + let mut proj_exprs = self + .schema + .fields + .iter() + .enumerate() + .map(|(idx, field)| { + let col = + Arc::new(Column::new(field.name(), idx)) as Arc; + (col, field.name().to_string()) + }) + .collect::>(); + // Project complex expression + let complex_proj_exprs = exprs + .iter() + .flat_map(|expr| { + if expr.as_any().is::() { + // Do not project column expressions + None + } else { + Some((expr.clone(), expr.to_string())) + } + }) + .collect::>(); + proj_exprs.extend(complex_proj_exprs); + ProjectionMapping::try_new(&proj_exprs, self.schema()).unwrap() + } + /// Projects argument `expr` according to `projection_mapping`, taking /// equivalences into account. /// @@ -1204,35 +1252,14 @@ impl EquivalenceProperties { /// definition of "partial permutation", see: /// /// + /// + /// This function applies an implicit projection to itself before calling `find_longest_permutation_helper`. + /// This enables us to consider complex expressions during analysis. pub fn find_longest_permutation( &self, exprs: &[Arc], ) -> (LexOrdering, Vec) { - let mut proj_exprs = self - .schema - .fields - .iter() - .enumerate() - .map(|(idx, field)| { - let col = - Arc::new(Column::new(field.name(), idx)) as Arc; - (col, field.name().to_string()) - }) - .collect::>(); - let complex_proj_exprs = exprs - .iter() - .flat_map(|expr| { - if expr.as_any().is::() { - // Do not project column expressions - None - } else { - Some((expr.clone(), expr.to_string())) - } - }) - .collect::>(); - proj_exprs.extend(complex_proj_exprs); - let projection_mapping = - ProjectionMapping::try_new(&proj_exprs, self.schema()).unwrap(); + let projection_mapping = self.implicit_projection_mapping(exprs); let projected = self.project(&projection_mapping, projection_mapping.output_schema()); projected.find_longest_permutation_helper( @@ -1240,6 +1267,7 @@ impl EquivalenceProperties { ) } + /// Helper function to calculate longest permutation. fn find_longest_permutation_helper( &self, exprs: &[Arc], From 8560c87dfd8e0c02f62e3dde557159565f0d9e68 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 6 Nov 2023 16:45:40 +0300 Subject: [PATCH 04/44] Minor changes --- 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 46979ad1795e..27c0cff453a0 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1951,8 +1951,8 @@ mod tests { table_data_with_properties.clone(), )?; let err_msg = format!( - "Error in test case requirement:{:?}, expected: {:?}, eq_properties.constants: {:?}, eq_properties.eq_group: {:?}, eq_properties.oeq_class: {:?}", - requirement, expected, eq_properties.constants, eq_properties.eq_group, eq_properties.oeq_class + "Error in test case requirement:{:?}, expected: {:?}", + requirement, expected ); // Check whether ordering_satisfy API result and // experimental result matches. From c31ec8cbf9c13b7be0574ea58af73a9b09623387 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 6 Nov 2023 18:20:04 +0300 Subject: [PATCH 05/44] Better projection support complex expression support --- .../src/physical_optimizer/enforce_sorting.rs | 217 +++ datafusion/physical-expr/src/equivalence.rs | 1460 +++++++++++++++-- .../physical-expr/src/sort_properties.rs | 114 +- .../physical-plan/src/aggregates/mod.rs | 2 + datafusion/physical-plan/src/projection.rs | 2 + datafusion/physical-plan/src/windows/mod.rs | 4 + 6 files changed, 1620 insertions(+), 179 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 4779ced44f1a..3f0ae4400f5a 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -2278,3 +2278,220 @@ mod tests { Ok(()) } } + +#[cfg(test)] +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_order() -> Result<()> { + let config = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::new_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 + 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 batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; + print_batches(&batches)?; + + let expected = vec![ + "GlobalLimitExec: skip=0, fetch=5", + " 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, actual, + "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + let expected = [ + "+--------+", + "| result |", + "+--------+", + "| 0 |", + "| 1 |", + "| 2 |", + "| 3 |", + "| 4 |", + "+--------+", + ]; + assert_batches_eq!(expected, &batches); + Ok(()) + } + + #[tokio::test] + async fn test_window() -> Result<()> { + let config = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::new_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 batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; + print_batches(&batches)?; + + let expected = 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, actual, + "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + let expected = [ + "+-----+-------------+--------------+", + "| c3 | sum1 | sum2 |", + "+-----+-------------+--------------+", + "| -86 | 2861911482 | 222089770060 |", + "| 13 | 5075947208 | 219227858578 |", + "| 125 | 8701233618 | 217013822852 |", + "| 123 | 11293564174 | 213388536442 |", + "| 97 | 14767488750 | 210796205886 |", + "+-----+-------------+--------------+", + ]; + assert_batches_eq!(expected, &batches); + Ok(()) + } + + #[tokio::test] + async fn test_aggregate() -> Result<()> { + let config = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::new_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 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 + 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 batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; + print_batches(&batches)?; + + let expected = vec![ + "GlobalLimitExec: skip=0, fetch=5", + " 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=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, 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, 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 expected = [ + "+---+-----------+", + "| a | last_col1 |", + "+---+-----------+", + "| 0 | 1 |", + "| 0 | 1 |", + "| 0 | 1 |", + "| 0 | 1 |", + "| 0 | 1 |", + "+---+-----------+", + ]; + assert_batches_eq!(expected, &batches); + Ok(()) + } +} diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 27c0cff453a0..82f106c9eb97 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 std::collections::{HashMap, HashSet}; +use std::collections::HashMap; use std::hash::Hash; use std::sync::Arc; @@ -34,6 +34,9 @@ use datafusion_common::{JoinSide, JoinType, Result}; use indexmap::map::Entry; use indexmap::IndexMap; +use itertools::Itertools; + +const PRINT_ON: bool = false; /// 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 @@ -472,31 +475,6 @@ impl EquivalenceGroup { None } - /// Projects `ordering` according to the given projection mapping. - /// If the resulting ordering is invalid after projection, returns `None`. - fn project_ordering( - &self, - mapping: &ProjectionMapping, - 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, - // 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 this equivalence group according to the given projection mapping. pub fn project(&self, mapping: &ProjectionMapping) -> Self { let projected_classes = self.iter().filter_map(|cls| { @@ -594,6 +572,24 @@ pub fn collapse_lex_req(input: LexRequirement) -> LexRequirement { output } +/// 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_ordering( + input: LexOrdering, + leading_ordering_exprs: &[Arc], +) -> LexOrdering { + let mut output = Vec::::new(); + for (idx, item) in input.into_iter().enumerate() { + if !(output.iter().any(|req| req.expr.eq(&item.expr)) + || (idx > 0 && physical_exprs_contains(leading_ordering_exprs, &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: /// @@ -679,6 +675,18 @@ impl OrderingEquivalenceClass { /// 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) { + let leading_ordering_exprs = self + .orderings + .iter() + .flat_map(|ordering| ordering.first().map(|sort_expr| sort_expr.expr.clone())) + .collect::>(); + self.orderings = self + .orderings + .iter() + .map(|ordering| { + collapse_lex_ordering(ordering.to_vec(), &leading_ordering_exprs) + }) + .collect::>(); let mut idx = 0; while idx < self.orderings.len() { let mut removal = self.orderings[idx].is_empty(); @@ -998,53 +1006,38 @@ impl EquivalenceProperties { /// This function applies an implicit projection to itself before calling `ordering_satisfy_requirement_helper`. /// This enables us to consider complex expressions during analysis. pub fn ordering_satisfy_requirement(&self, reqs: LexRequirementRef) -> bool { - let exprs = reqs - .iter() - .map(|sort_req| sort_req.expr.clone()) - .collect::>(); - let projection_mapping = self.implicit_projection_mapping(&exprs); - let projected_eqs = - self.project(&projection_mapping, projection_mapping.output_schema()); - if let Some(projected_reqs) = projection_mapping.project_lex_reqs(reqs) { - projected_eqs.ordering_satisfy_requirement_helper(&projected_reqs) - } else { - false + let projected_eqs = self.clone(); + projected_eqs.ordering_satisfy_requirement_helper2(reqs) + } + + fn ordering_satisfy_requirement_helper3( + &self, + req: &PhysicalSortRequirement, + ) -> bool { + let expr_ordering = self.get_expr_ordering(req.expr.clone()); + let ExprOrdering { expr, state, .. } = expr_ordering; + match state { + SortProperties::Ordered(options) => { + let sort_expr = PhysicalSortExpr { expr, options }; + sort_expr.satisfy(req, self.schema()) + } + SortProperties::Singleton => true, + SortProperties::Unordered => false, } } /// Helper function to check whether the given sort requirements are satisfied by any of the /// existing orderings. - fn ordering_satisfy_requirement_helper(&self, reqs: LexRequirementRef) -> bool { + fn ordering_satisfy_requirement_helper2(mut self, reqs: LexRequirementRef) -> bool { // 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; - } - } + for normalized_req in normalized_reqs { + if !self.ordering_satisfy_requirement_helper3(&normalized_req) { + return false; } + self = self.add_constants(std::iter::once(normalized_req.expr)); } - indices.len() == normalized_reqs.len() + true } /// Checks whether the `given`` sort requirements are equal or more specific @@ -1205,39 +1198,342 @@ impl EquivalenceProperties { self.eq_group.project_expr(projection_mapping, expr) } - /// Projects the equivalences within according to `projection_mapping` - /// and `output_schema`. - pub fn project( + fn get_leading_orderings( &self, - projection_mapping: &ProjectionMapping, - output_schema: SchemaRef, - ) -> Self { - let mut projected_orderings = self - .oeq_class + ignore_exprs: &[Arc], + ) -> Vec { + let mut leading_orderings = vec![]; + let normalized_ignore_exprs = + self.eq_group.normalize_exprs(ignore_exprs.to_vec()); + for ordering in self.normalized_oeq_class().iter() { + for sort_expr in ordering { + if !expr_consists_of_ignored_exprs( + &normalized_ignore_exprs, + &sort_expr.expr, + ) { + leading_orderings.push(sort_expr.clone()); + break; + } + } + } + leading_orderings + } + + /// Projects `ordering` according to the given projection mapping. + /// If the resulting ordering is invalid after projection, returns `None`. + fn project_ordering_helper( + &self, + mapping: &ProjectionMapping, + 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, + // the result should be [a ASC], not [a ASC, c ASC], even if column c is + // valid after projection. + let result = ordering .iter() - .filter_map(|order| self.eq_group.project_ordering(projection_mapping, order)) + .map_while(|sort_expr| { + self.project_expr(&sort_expr.expr, mapping) + .map(|expr| PhysicalSortExpr { + expr, + options: sort_expr.options, + }) + }) .collect::>(); - for (source, target) in projection_mapping.iter() { - let expr_ordering = ExprOrdering::new(source.clone()) - .transform_up(&|expr| update_ordering(expr, self)) - .unwrap(); - if let SortProperties::Ordered(options) = expr_ordering.state { - // Push new ordering to the state. - projected_orderings.push(vec![PhysicalSortExpr { + (!result.is_empty()).then_some(result) + } + + // TODO: Add a stop signal as return value. + fn calc_ordered_exprs( + &self, + mapping: &ProjectionMapping, + ignored_exprs: &[Arc], + constants: &[Arc], + ) -> Vec> { + let mut new_orderings = vec![]; + let leading_orderings = self.get_leading_orderings(ignored_exprs); + for (source, target) in mapping.iter() { + // let expr_ordering = ExprOrdering::new_construction( + // source.clone(), + // &self.eq_group, + // &leading_orderings, + // constants, + // ); + let expr_ordering = self.get_expr_ordering(source.clone()); + if PRINT_ON { + println!("expr_ordering: {expr_ordering:?}"); + println!("self.constants: {:?}", self.constants); + } + let sort_options = match expr_ordering.state { + SortProperties::Ordered(options) => Some(options), + // Assign an arbitrary ordering direction to singleton columns. + SortProperties::Singleton => Some(SortOptions { + descending: false, + nulls_first: false, + }), + SortProperties::Unordered => None, + }; + if let Some(options) = sort_options { + let sort_expr = PhysicalSortExpr { expr: target.clone(), options, - }]); + }; + let new_ordering = vec![sort_expr.clone()]; + if let Some(leaf_orderings) = + expr_ordering.leaf_orderings(&leading_orderings) + { + if let Some(projected_leaf_orderings) = + self.project_ordering_helper(mapping, &leaf_orderings) + { + // All of the leaves are projected. + if projected_leaf_orderings.len() == leaf_orderings.len() { + // TODO: If expression is known to be strictly monothonic. Then we do not need to append leaf ordering expressions + for suffix in projected_leaf_orderings + .iter() + .permutations(projected_leaf_orderings.len()) + { + let mut new_ordering = vec![sort_expr.clone()]; + let suffix = suffix.into_iter().cloned(); + new_ordering.extend(suffix); + let new_ordering = + collapse_lex_ordering(new_ordering, ignored_exprs); + if !new_orderings.contains(&new_ordering) { + new_orderings.push(new_ordering); + } + } + // new_ordering.extend(projected_leaf_orderings); + } else { + if PRINT_ON { + println!( + "send stop signal, expr_ordering: {:?}", + expr_ordering + ); + } + // TODO: send stop signal. + } + } else { + if PRINT_ON { + println!( + "send stop signal, expr_ordering: {:?}", + expr_ordering + ); + } + // TODO: send stop signal. + } + } + let new_ordering = collapse_lex_ordering(new_ordering, ignored_exprs); + if !new_orderings.contains(&new_ordering) { + new_orderings.push(new_ordering); + } + } + } + for leading_ordering in leading_orderings { + if let Some(projected_leading_ordering) = + self.project_ordering_helper(mapping, &[leading_ordering]) + { + if !new_orderings.contains(&projected_leading_ordering) { + new_orderings.push(projected_leading_ordering); + } + } + } + if PRINT_ON { + for new_ordering in &new_orderings { + println!("new_ordering: {new_ordering:?}"); + } + } + new_orderings.retain(|ordering| !ordering.is_empty()); + new_orderings + } + + /// Projects `ordering` according to the given projection mapping. + /// If the resulting ordering is invalid after projection, returns `None`. + fn project_ordering( + mut self, + mapping: &ProjectionMapping, + 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, + // the result should be [a ASC], not [a ASC, c ASC], even if column c is + // valid after projection. + let mut orderings = vec![vec![]]; + let mut ignored_exprs = vec![]; + let mut constants = self.constants.clone(); + for sort_expr in ordering.iter() { + let new_orderings = + self.calc_ordered_exprs(mapping, &ignored_exprs, &constants); + let mut projected_ignored_exprs = if let Some(ignored_exprs) = ignored_exprs + .iter() + .map(|expr| self.eq_group.project_expr(mapping, expr)) + .collect::>>() + { + ignored_exprs + } else { + // At least one of the ignored_exprs cannot be projected. Hence continuing the iteration is not guaranteed to consider all referred columns. + // Stop iteration. + break; + }; + for (source, target) in mapping.iter() { + if expr_consists_of_ignored_exprs(&ignored_exprs, source) + && !physical_exprs_contains(&projected_ignored_exprs, source) + { + projected_ignored_exprs.push(target.clone()); + } + } + + if PRINT_ON { + for ordering in &orderings { + println!("ordering: {ordering:?}"); + } + } + + if new_orderings.is_empty() { + break; + } else { + let relevant_indices = orderings + .iter() + .enumerate() + .flat_map(|(idx, ordering)| { + if ordering_consists_of_ignored_exprs( + &projected_ignored_exprs, + ordering, + ) { + Some(idx) + } else { + None + } + }) + .collect::>(); + let n_rep = new_orderings.len(); + let orderings_len = orderings.len(); + for relevant_idx in &relevant_indices { + for _rep_idx in 0..n_rep { + orderings.push(orderings[*relevant_idx].clone()); + } + } + + let new_indices = (orderings_len + ..orderings_len + (n_rep) * relevant_indices.len()) + .collect::>(); + + for (iter_idx, ordering_idx) in new_indices.into_iter().enumerate() { + let mod_idx = iter_idx % new_orderings.len(); + let new_ordering = &new_orderings[mod_idx]; + orderings[ordering_idx].extend(new_ordering.to_vec()); + } + } + let leaves = leaf_exprs(sort_expr.expr.clone()); + let leaves = self.eq_group.normalize_exprs(leaves); + ignored_exprs.extend(leaves); + ignored_exprs.push(sort_expr.expr.clone()); + constants.push(sort_expr.expr.clone()); + self = self.add_constants(std::iter::once(sort_expr.expr.clone())); + deduplicate_physical_exprs(&mut ignored_exprs); + if PRINT_ON { + for ordering in &orderings { + println!("updated ordering: {ordering:?}"); + } + } + orderings = orderings + .into_iter() + .map(|ordering| self.eq_group.normalize_sort_exprs(&ordering)) + .collect(); + let oeq_class = OrderingEquivalenceClass::new(orderings); + orderings = oeq_class.orderings; + if PRINT_ON { + for ordering in &orderings { + println!("updated ordering2: {ordering:?}"); + } + } + } + let leading_orderings = self.get_leading_orderings(&[]); + let normalized_leading_orderings = + self.eq_group.normalize_sort_exprs(&leading_orderings); + let projected_leading_orderings = self + .project_ordering_helper(mapping, &normalized_leading_orderings) + .unwrap_or_default(); + let normalized_leading_exprs = projected_leading_orderings + .into_iter() + .map(|sort_expr| sort_expr.expr) + .collect::>(); + let leading_ordering_exprs = orderings + .iter() + .flat_map(|ordering| ordering.first().map(|sort_expr| sort_expr.expr.clone())) + .collect::>(); + + if PRINT_ON { + println!("normalized_leading_exprs: {:?}", normalized_leading_exprs); + println!( + "normalized_leading_orderings: {:?}", + normalized_leading_orderings + ); + println!("leading_ordering_exprs: {:?}", leading_ordering_exprs); + } + orderings = orderings + .into_iter() + .map(|ordering| { + let collapsed = collapse_lex_ordering(ordering, &leading_ordering_exprs); + let normalized = self.eq_group.normalize_sort_exprs(&collapsed); + // println!("normalized: {:?}", normalized); + // println!("normalized_leading_exprs: {:?}", normalized_leading_exprs); + // println!("leading_ordering_exprs: {:?}", leading_ordering_exprs); + normalized + .into_iter() + .enumerate() + .flat_map(|(idx, sort_expr)| { + if idx > 0 + && physical_exprs_contains( + &leading_ordering_exprs, + &sort_expr.expr, + ) + { + None + } else { + Some(sort_expr) + } + }) + .collect::>() + }) + .collect(); + if PRINT_ON { + println!("leading_ordering_exprs: {:?}", leading_ordering_exprs); + println!("self.constants: {:?}", self.constants); + println!("self.eq_group: {:?}", self.eq_group); + for ordering in &orderings { + println!("updated ordering3: {ordering:?}"); } } - let target_constants = self + orderings.retain(|ordering| !ordering.is_empty()); + (!orderings.is_empty()).then_some(orderings) + } + + /// Projects the equivalences within according to `projection_mapping` + /// and `output_schema`. + pub fn project( + mut self, + projection_mapping: &ProjectionMapping, + output_schema: SchemaRef, + ) -> Self { + let projected_constants = self .constants .iter() .flat_map(|constant| projection_mapping.target_expr(constant)) .collect::>(); + let projected_eq_group = self.eq_group.project(projection_mapping); + + let projected_orderings = self + .oeq_class + .iter() + .filter_map(|order| self.clone().project_ordering(projection_mapping, order)) + .flatten() + .collect::>(); + Self { - eq_group: self.eq_group.project(projection_mapping), + eq_group: projected_eq_group, oeq_class: OrderingEquivalenceClass::new(projected_orderings), - constants: target_constants, + constants: projected_constants, schema: output_schema, } } @@ -1260,8 +1556,9 @@ impl EquivalenceProperties { exprs: &[Arc], ) -> (LexOrdering, Vec) { let projection_mapping = self.implicit_projection_mapping(exprs); - let projected = - self.project(&projection_mapping, projection_mapping.output_schema()); + let projected = self + .clone() + .project(&projection_mapping, projection_mapping.output_schema()); projected.find_longest_permutation_helper( &projection_mapping.target_exprs(exprs).unwrap_or_default(), ) @@ -1305,6 +1602,66 @@ impl EquivalenceProperties { }) .unzip() } + + fn is_expr_constant(&self, expr: &Arc) -> bool { + let normalized_constants = self.eq_group.normalize_exprs(self.constants.to_vec()); + let normalized_expr = self.eq_group.normalize_expr(expr.clone()); + physical_exprs_contains(&normalized_constants, &normalized_expr) + } + + pub fn get_expr_ordering(&self, expr: Arc) -> ExprOrdering { + let expr_ordering = ExprOrdering::new(expr.clone()); + expr_ordering + .transform_up(&|expr| update_ordering(expr, self)) + .unwrap() + } +} + +fn expr_consists_of_ignored_exprs( + ignored_exprs: &[Arc], + expr: &Arc, +) -> bool { + if physical_exprs_contains(ignored_exprs, expr) { + return true; + } + let children = expr.children(); + if children.is_empty() { + false + } else { + children + .iter() + .all(|child| expr_consists_of_ignored_exprs(ignored_exprs, child)) + } +} + +fn ordering_consists_of_ignored_exprs( + ignored_exprs: &[Arc], + ordering: &[PhysicalSortExpr], +) -> bool { + ordering + .iter() + .all(|sort_expr| expr_consists_of_ignored_exprs(ignored_exprs, &sort_expr.expr)) +} + +fn leaf_exprs(expr: Arc) -> Vec> { + let mut result = vec![]; + leaf_exprs_helper(expr, &mut result); + result +} + +fn leaf_exprs_helper( + expr: Arc, + result: &mut Vec>, +) { + if expr.children().is_empty() { + if !physical_exprs_contains(result, &expr) { + result.push(expr); + } + } else { + expr.children() + .into_iter() + .for_each(|child| leaf_exprs_helper(child, result)); + } } /// Calculate ordering equivalence properties for the given join operation. @@ -1427,8 +1784,11 @@ fn update_ordering( // We have a Column, which is one of the two possible leaf node types: let eq_group = &eq_properties.eq_group; let normalized_expr = eq_group.normalize_expr(node.expr.clone()); - let oeq_class = &eq_properties.oeq_class; - if let Some(options) = oeq_class.get_options(&normalized_expr) { + let oeq_class = &eq_properties.normalized_oeq_class(); + if eq_properties.is_expr_constant(&normalized_expr) { + node.state = SortProperties::Singleton; + Ok(Transformed::Yes(node)) + } else if let Some(options) = oeq_class.get_options(&normalized_expr) { node.state = SortProperties::Ordered(options); Ok(Transformed::Yes(node)) } else { @@ -1447,16 +1807,18 @@ mod tests { use std::sync::Arc; use super::*; - use crate::expressions::{col, lit, BinaryExpr, Column}; + use crate::expressions::{col, lit, BinaryExpr, Column, Literal}; use crate::physical_expr::{physical_exprs_bag_equal, 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 datafusion_expr::Operator; + use arrow_array::{ArrayRef, Float64Array, RecordBatch, UInt32Array}; + use arrow_schema::{Fields, SortOptions, TimeUnit}; + use datafusion_common::{Result, ScalarValue}; + use datafusion_expr::{BuiltinScalarFunction, Operator}; + use crate::execution_props::ExecutionProps; + use crate::functions::create_physical_expr; use itertools::{izip, Itertools}; use rand::rngs::StdRng; use rand::seq::SliceRandom; @@ -1521,12 +1883,12 @@ mod tests { // 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 a = Field::new("a", DataType::Float64, true); + let b = Field::new("b", DataType::Float64, true); + let c = Field::new("c", DataType::Float64, true); + let d = Field::new("d", DataType::Float64, true); + let e = Field::new("e", DataType::Float64, true); + let f = Field::new("f", DataType::Float64, true); let schema = Arc::new(Schema::new(vec![a, b, c, d, e, f])); Ok(schema) @@ -1615,6 +1977,29 @@ mod tests { .collect() } + // Convert each tuple to PhysicalSortExpr + fn convert_to_sort_exprs_owned( + in_data: &[(Arc, SortOptions)], + ) -> Vec { + in_data + .iter() + .map(|(expr, options)| PhysicalSortExpr { + expr: (*expr).clone(), + options: *options, + }) + .collect::>() + } + + // Convert each inner tuple to PhysicalSortExpr + fn convert_to_orderings_owned( + orderings: &[Vec<(Arc, SortOptions)>], + ) -> Vec> { + orderings + .iter() + .map(|sort_exprs| convert_to_sort_exprs_owned(sort_exprs)) + .collect() + } + #[test] fn add_equal_conditions_test() -> Result<()> { let schema = Arc::new(Schema::new(vec![ @@ -1951,8 +2336,8 @@ mod tests { table_data_with_properties.clone(), )?; let err_msg = format!( - "Error in test case requirement:{:?}, expected: {:?}", - requirement, expected + "Error in test case requirement:{:?}, expected: {:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}", + requirement, expected, eq_properties.oeq_class, eq_properties.eq_group, eq_properties.constants ); // Check whether ordering_satisfy API result and // experimental result matches. @@ -1970,31 +2355,373 @@ 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 options = SortOptions { + fn test_ordering_satisfy_with_equivalence_complex_random() -> Result<()> { + const N_RANDOM_SCHEMA: usize = 100; + const N_ELEMENTS: usize = 125; + const N_DISTINCT: usize = 5; + const SORT_OPTIONS: SortOptions = SortOptions { descending: false, nulls_first: false, }; - // a=c (e.g they are aliases). - let mut eq_properties = EquivalenceProperties::new(test_schema); - eq_properties.add_equal_conditions(col_a, col_c); - - let orderings = vec![ - vec![(col_a, options)], - vec![(col_e, options)], - vec![(col_d, options), (col_f, options)], - ]; - let orderings = convert_to_orderings(&orderings); - - // Column [a ASC], [e ASC], [d ASC, f ASC] are all valid orderings for the schema. - eq_properties.add_new_orderings(orderings); + + for seed in 0..N_RANDOM_SCHEMA { + // Create a random schema with random properties + 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_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; + // print_batches(&[table_data_with_properties.clone()])?; + let exp_fn = create_physical_expr( + &BuiltinScalarFunction::Floor, + &[col("a", &test_schema)?], + &test_schema, + &ExecutionProps::default(), + )?; + let a_plus_b = Arc::new(BinaryExpr::new( + col("a", &test_schema)?, + Operator::Plus, + col("b", &test_schema)?, + )) as Arc; + let 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)?, + exp_fn, + a_plus_b, + ]; + + for n_req in 0..=exprs.len() { + for exprs in exprs.iter().combinations(n_req) { + let requirement = exprs + .into_iter() + .map(|expr| PhysicalSortExpr { + expr: expr.clone(), + 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:{:?}, expected: {:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}", + requirement, expected, eq_properties.oeq_class, eq_properties.eq_group, eq_properties.constants + ); + // Check whether ordering_satisfy API result and + // experimental result matches. + + // assert!( + // !(eq_properties.ordering_satisfy(&requirement) == true + // && expected == false), + // "{}", + // err_msg + // ); + + assert_eq!( + eq_properties.ordering_satisfy(&requirement), + (expected | false), + "{}", + err_msg + ); + } + } + } + + Ok(()) + } + + #[test] + fn test_ordering_satisfy_err_cases() -> 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 floor_a = &create_physical_expr( + &BuiltinScalarFunction::Floor, + &[col("a", &test_schema)?], + &test_schema, + &ExecutionProps::default(), + )?; + let exp_a = &create_physical_expr( + &BuiltinScalarFunction::Exp, + &[col("a", &test_schema)?], + &test_schema, + &ExecutionProps::default(), + )?; + let a_plus_b = Arc::new(BinaryExpr::new( + col_a.clone(), + Operator::Plus, + col_b.clone(), + )) as Arc; + let options = SortOptions { + descending: false, + nulls_first: false, + }; + + let test_cases = vec![ + // ------------ TEST CASE 1 ------------ + ( + // orderings + vec![ + // [a ASC, d ASC, b ASC] + vec![(col_a, options), (col_d, options), (col_b, options)], + // [c ASC] + vec![(col_c, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [a ASC, b ASC], requirement is not satisfied. + vec![(col_a, options), (col_b, options)], + // expected: requirement is not satisfied. + false, + ), + // ------------ TEST CASE 2 ------------ + ( + // orderings + vec![ + // [a ASC, c ASC, b ASC] + vec![(col_a, options), (col_c, options), (col_b, options)], + // [d ASC] + vec![(col_d, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [floor(a) ASC], + vec![(floor_a, options)], + // expected: requirement is satisfied. + true, + ), + // ------------ TEST CASE 3 ------------ + ( + // orderings + vec![ + // [a ASC, c ASC, b ASC] + vec![(col_a, options), (col_c, options), (col_b, options)], + // [d ASC] + vec![(col_d, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [a ASC, c ASC, a+b ASC], + vec![(col_a, options), (col_c, options), (&a_plus_b, options)], + // expected: requirement is satisfied. + true, + ), + // ------------ TEST CASE 4 ------------ + ( + // orderings + vec![ + // [a ASC, b ASC, c ASC, d ASC] + vec![ + (col_a, options), + (col_b, options), + (col_c, options), + (col_d, options), + ], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [floor(a) ASC, a+b ASC], + vec![(floor_a, options), (&a_plus_b, options)], + // expected: requirement is satisfied. + false, + ), + // ------------ TEST CASE 5 ------------ + ( + // orderings + vec![ + // [a ASC, b ASC, c ASC, d ASC] + vec![ + (col_a, options), + (col_b, options), + (col_c, options), + (col_d, options), + ], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [exp(a) ASC, a+b ASC], + vec![(exp_a, options), (&a_plus_b, options)], + // expected: requirement is not satisfied. + // TODO: If we know that exp function is 1-to-1 function. + // we could have deduced that above requirement is satisfied. + false, + ), + // ------------ TEST CASE 6 ------------ + ( + // orderings + vec![ + // [a ASC, d ASC, b ASC] + vec![(col_a, options), (col_d, options), (col_b, options)], + // [c ASC] + vec![(col_c, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [a ASC, d ASC, floor(a) ASC], + vec![(col_a, options), (col_d, options), (floor_a, options)], + // expected: requirement is satisfied. + true, + ), + // ------------ TEST CASE 7 ------------ + ( + // orderings + vec![ + // [a ASC, c ASC, b ASC] + vec![(col_a, options), (col_c, options), (col_b, options)], + // [d ASC] + vec![(col_d, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [a ASC, floor(a) ASC, a + b ASC], + vec![(col_a, options), (floor_a, options), (&a_plus_b, options)], + // expected: requirement is not satisfied. + false, + ), + // ------------ TEST CASE 8 ------------ + ( + // orderings + vec![ + // [a ASC, b ASC, c ASC] + vec![(col_a, options), (col_b, options), (col_c, options)], + // [d ASC] + vec![(col_d, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [a ASC, c ASC, floor(a) ASC, a + b ASC], + vec![ + (col_a, options), + (col_c, options), + (&floor_a, options), + (&a_plus_b, options), + ], + // expected: requirement is not satisfied. + false, + ), + // ------------ TEST CASE 9 ------------ + ( + // orderings + vec![ + // [a ASC, b ASC, c ASC, d ASC] + vec![ + (col_a, options), + (col_b, options), + (col_c, options), + (col_d, options), + ], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [a ASC, b ASC, c ASC, floor(a) ASC], + vec![ + (col_a, options), + (col_b, options), + (&col_c, options), + (&floor_a, options), + ], + // expected: requirement is satisfied. + true, + ), + // ------------ TEST CASE 10 ------------ + ( + // orderings + vec![ + // [d ASC, b ASC] + vec![(col_d, options), (col_b, options)], + // [c ASC, a ASC] + vec![(col_c, options), (col_a, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [c ASC, d ASC, a + b ASC], + vec![(col_c, options), (col_d, options), (&a_plus_b, options)], + // expected: requirement is satisfied. + true, + ), + ]; + + for (orderings, eq_group, constants, reqs, expected) in test_cases { + let err_msg = + format!("error in test orderings: {orderings:?}, eq_group: {eq_group:?}, constants: {constants:?}, reqs: {reqs:?}, expected: {expected:?}"); + let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); + let orderings = convert_to_orderings(&orderings); + eq_properties.add_new_orderings(orderings); + let eq_group = eq_group + .into_iter() + .map(|eq_class| eq_class.into_iter().cloned().collect::>()) + .collect::>(); + let eq_group = EquivalenceGroup::new(eq_group); + eq_properties.add_equivalence_group(eq_group); + + let constants = constants.into_iter().cloned(); + eq_properties = eq_properties.add_constants(constants); + + let reqs = convert_to_sort_exprs(&reqs); + assert_eq!( + eq_properties.ordering_satisfy(&reqs), + expected, + "{}", + err_msg + ); + } + + Ok(()) + } + + #[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 options = SortOptions { + descending: false, + nulls_first: false, + }; + // a=c (e.g they are aliases). + let mut eq_properties = EquivalenceProperties::new(test_schema); + eq_properties.add_equal_conditions(col_a, col_c); + + let orderings = vec![ + vec![(col_a, options)], + vec![(col_e, options)], + vec![(col_d, options), (col_f, options)], + ]; + let orderings = convert_to_orderings(&orderings); + + // Column [a ASC], [e ASC], [d ASC, f ASC] are all valid orderings for the schema. + 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. @@ -2308,13 +3035,16 @@ mod tests { let mut columns = batch.columns().to_vec(); // Create a new unique column - let n_row = batch.num_rows() as u64; - let unique_col = Arc::new(UInt64Array::from_iter_values(0..n_row)) as ArrayRef; + let n_row = batch.num_rows(); + let vals: Vec = (0..n_row).collect::>(); + let vals: Vec = vals.into_iter().map(|val| val as f64).collect(); + let unique_col = Arc::new(Float64Array::from_iter_values(vals)) 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 unique_field = + Arc::new(Field::new(unique_col_name, DataType::Float64, false)); let fields: Vec<_> = original_schema .fields() .iter() @@ -2335,13 +3065,13 @@ mod tests { // 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(), + .map(|order_expr| { + let expr_result = order_expr.expr.evaluate(&new_batch).unwrap(); + let values = expr_result.into_array(new_batch.num_rows()); + SortColumn { + values, options: Some(order_expr.options), - }) + } }) .collect(); @@ -2384,18 +3114,18 @@ mod tests { // 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) + let values: Vec = (0..num_elems) + .map(|_| rng.gen_range(0..max_val) as f64 / 2.0) .collect(); - Arc::new(UInt64Array::from_iter_values(values)) + Arc::new(Float64Array::from_iter_values(values)) }; // Fill constant columns 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 = - Arc::new(UInt64Array::from_iter_values(vec![0; n_elem])) as ArrayRef; + let arr = Arc::new(Float64Array::from_iter_values(vec![0 as f64; n_elem])) + as ArrayRef; schema_vec[idx] = Some(arr); } @@ -2741,9 +3471,9 @@ mod tests { ( vec![col_d, col_e, col_b], vec![ + (col_e, option_desc), (col_d, option_asc), (col_b, option_asc), - (col_e, option_desc), ], ), // TEST CASE 4 @@ -2818,11 +3548,14 @@ 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, &eq_properties))?; + let leading_orderings = eq_properties + .oeq_class() + .iter() + .flat_map(|ordering| ordering.first().cloned()) + .collect::>(); + let expr_ordering = eq_properties.get_expr_ordering(expr.clone()); let err_msg = format!( - "expr:{:?}, expected: {:?}, actual: {:?}", + "expr:{:?}, expected: {:?}, actual: {:?}, leading_orderings: {leading_orderings:?}", expr, expected, expr_ordering.state ); assert_eq!(expr_ordering.state, expected, "{}", err_msg); @@ -2990,56 +3723,427 @@ mod tests { } #[test] - fn project_empty_output_ordering() -> Result<()> { + fn project_orderings() -> Result<()> { let schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::Int32, true), Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + Field::new("ts", DataType::Timestamp(TimeUnit::Nanosecond, None), true), ])); - let mut eq_properties = EquivalenceProperties::new(schema.clone()); - let ordering = vec![PhysicalSortExpr { - expr: col("b", &schema)?, - options: SortOptions::default(), - }]; - eq_properties.add_new_orderings([ordering]); - // b as b_new, a as a_new - let proj_exprs = vec![ - (col("b", &schema)?, "b_new".to_string()), - (col("a", &schema)?, "a_new".to_string()), + 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_ts = &col("ts", &schema)?; + let interval = Arc::new(Literal::new(ScalarValue::IntervalDayTime(Some(2)))) + as Arc; + let date_bin_func = &create_physical_expr( + &BuiltinScalarFunction::DateBin, + &[interval, col_ts.clone()], + &schema, + &ExecutionProps::default(), + )?; + let a_plus_b = Arc::new(BinaryExpr::new( + col_a.clone(), + Operator::Plus, + col_b.clone(), + )) as Arc; + let b_plus_d = Arc::new(BinaryExpr::new( + col_b.clone(), + Operator::Plus, + col_d.clone(), + )) as Arc; + let c_plus_d = Arc::new(BinaryExpr::new( + col_c.clone(), + Operator::Plus, + col_d.clone(), + )) as Arc; + + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + + let test_cases = vec![ + // ---------- TEST CASE 1 ------------ + ( + // orderings + vec![ + // [b ASC] + vec![(col_b, option_asc)], + ], + // projection exprs + vec![(col_b, "b_new".to_string()), (col_a, "a_new".to_string())], + // expected + vec![ + // [b_new ASC] + vec![("b_new", option_asc)], + ], + ), + // ---------- TEST CASE 2 ------------ + ( + // orderings + vec![ + // empty ordering + ], + // projection exprs + vec![(col_c, "c_new".to_string()), (col_b, "b_new".to_string())], + // expected + vec![ + // no ordering at the output + ], + ), + // ---------- TEST CASE 3 ------------ + ( + // orderings + vec![ + // [ts ASC] + vec![(col_ts, option_asc)], + ], + // projection exprs + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (col_ts, "ts_new".to_string()), + (date_bin_func, "date_bin_res".to_string()), + ], + // expected + vec![ + // [date_bin_res ASC] + vec![("date_bin_res", option_asc)], + // [ts_new ASC] + vec![("ts_new", option_asc)], + ], + ), + // ---------- TEST CASE 4 ------------ + ( + // orderings + vec![ + // [a ASC, ts ASC] + vec![(col_a, option_asc), (col_ts, option_asc)], + // [b ASC, ts ASC] + vec![(col_b, option_asc), (col_ts, option_asc)], + ], + // projection exprs + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (col_ts, "ts_new".to_string()), + (date_bin_func, "date_bin_res".to_string()), + ], + // expected + vec![ + // [a_new ASC, ts_new ASC] + vec![("a_new", option_asc), ("ts_new", option_asc)], + // [a_new ASC, date_bin_res ASC, ts_new ASC] + vec![ + ("a_new", option_asc), + ("date_bin_res", option_asc), + ("ts_new", option_asc), + ], + // [b_new ASC, ts_new ASC] + vec![("b_new", option_asc), ("ts_new", option_asc)], + // [b_new ASC, date_bin_res ASC, ts_new ASC] + vec![ + ("b_new", option_asc), + ("date_bin_res", option_asc), + ("ts_new", option_asc), + ], + ], + ), + // ---------- TEST CASE 5 ------------ + ( + // orderings + vec![ + // [a + b ASC] + vec![(&a_plus_b, option_asc)], + ], + // projection exprs + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (&a_plus_b, "a+b".to_string()), + ], + // expected + vec![ + // [a + b ASC] + vec![("a+b", option_asc)], + ], + ), + // ---------- TEST CASE 6 ------------ + ( + // orderings + vec![ + // [a + b ASC, c ASC] + vec![(&a_plus_b, option_asc), (&col_c, option_asc)], + ], + // projection exprs + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (col_c, "c_new".to_string()), + (&a_plus_b, "a+b".to_string()), + ], + // expected + vec![ + // [a + b ASC, c_new ASC] + vec![("a+b", option_asc), ("c_new", option_asc)], + ], + ), + // ------- TEST CASE 7 ---------- + ( + // orderings + vec![ + // [a ASC, ts ASC] + vec![(&col_a, option_asc), (&col_ts, option_asc)], + // [b ASC, ts ASC] + vec![(&col_b, option_asc), (&col_ts, option_asc)], + ], + // projection exprs + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (col_ts, "ts_new".to_string()), + (date_bin_func, "date_bin_res".to_string()), + ], + // expected + vec![ + // [a_new ASC, ts_new ASC] + vec![("a_new", option_asc), ("ts_new", option_asc)], + // [a_new ASC, date_bin_res ASC, ts_new ASC] + vec![ + ("a_new", option_asc), + ("date_bin_res", option_asc), + ("ts_new", option_asc), + ], + // [b_new ASC, ts_new ASC] + vec![("b_new", option_asc), ("ts_new", option_asc)], + // [b_new ASC, date_bin_res ASC, ts_new ASC] + vec![ + ("b_new", option_asc), + ("date_bin_res", option_asc), + ("ts_new", option_asc), + ], + ], + ), + // ------- TEST CASE 8 ---------- + ( + vec![ + // [a ASC, b ASC, c ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + // [a ASC, d ASC] + vec![(col_a, option_asc), (col_d, option_asc)], + ], + // b as b_new, a as a_new, d as d_new b+d + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (col_d, "d_new".to_string()), + (&b_plus_d, "b+d".to_string()), + ], + // expected, This expected may be missing + vec![ + // [a_new ASC, b_new ASC] + vec![("a_new", option_asc), ("b_new", option_asc)], + // [a_new ASC, d_new ASC] + vec![("a_new", option_asc), ("d_new", option_asc)], + // [a_new ASC, d_new ASC] + // TODO: In below case lexicographical argument ordering may be unnecessary for 1-to-1 functions. + vec![ + ("a_new", option_asc), + ("b+d", option_asc), + ("b_new", option_asc), + ("d_new", option_asc), + ], + vec![ + ("a_new", option_asc), + ("b+d", option_asc), + ("d_new", option_asc), + ("b_new", option_asc), + ], + ], + ), + // ------- TEST CASE 9 ---------- + ( + // orderings + vec![ + // [b+d ASC] + vec![(&b_plus_d, option_asc)], + ], + // proj exprs + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (col_d, "d_new".to_string()), + (&b_plus_d, "b+d".to_string()), + ], + // expected + vec![ + // [b+d ASC] + vec![("b+d", option_asc)], + ], + ), + // ------- TEST CASE 10 ---------- + ( + // orderings + vec![ + // [a ASC, d ASC, b ASC] + vec![ + (col_a, option_asc), + (col_d, option_asc), + (col_b, option_asc), + ], + // [c ASC] + vec![(col_c, option_asc)], + ], + // proj exprs + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (col_d, "d_new".to_string()), + (col_c, "c_new".to_string()), + ], + // expected + vec![ + // [a_new ASC, d_new ASC, b_new ASC] + vec![ + ("a_new", option_asc), + ("d_new", option_asc), + ("b_new", option_asc), + ], + // [c_new ASC] + vec![("c_new", option_asc)], + ], + ), + // ------- TEST CASE 11 ---------- + ( + vec![ + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + // [a ASC, d ASC] + vec![(col_a, option_asc), (col_d, option_asc)], + ], + // proj exprs + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (col_c, "c_new".to_string()), + (&c_plus_d, "c+d".to_string()), + ], + // expected + vec![ + // [a_new ASC, b_new ASC, c_new ASC] + vec![ + ("a_new", option_asc), + ("b_new", option_asc), + ("c_new", option_asc), + ], + // [a_new ASC, b_new ASC, c+d ASC] + vec![ + ("a_new", option_asc), + ("b_new", option_asc), + ("c+d", option_asc), + ], + ], + ), + // ------- TEST CASE 12 ---------- + ( + // orderings + vec![ + // [a ASC, b ASC] + vec![(&col_a, option_asc), (&col_b, option_asc)], + // [a ASC, d ASC] + vec![(&col_a, option_asc), (&col_d, option_asc)], + ], + // proj exprs + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (&b_plus_d, "b+d".to_string()), + ], + // expected, This expected may be missing + vec![ + // [a_new ASC, b_new ASC] + vec![("a_new", option_asc), ("b_new", option_asc)], + // [a_new ASC, b + d ASC] + vec![("a_new", option_asc), ("b+d", option_asc)], + ], + ), ]; - let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &schema)?; - let orderings = eq_properties - .project(&projection_mapping, projection_mapping.output_schema()) - .oeq_class() - .output_ordering() - .unwrap_or_default(); + for (orderings, proj_exprs, expected) in test_cases { + let mut eq_properties = EquivalenceProperties::new(schema.clone()); - assert_eq!( - vec![PhysicalSortExpr { - expr: Arc::new(Column::new("b_new", 0)), - options: SortOptions::default(), - }], - orderings - ); + let orderings = convert_to_orderings(&orderings); + eq_properties.add_new_orderings(orderings); + let proj_exprs = proj_exprs + .into_iter() + .map(|(expr, name)| (expr.clone(), name)) + .collect::>(); + let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &schema)?; + let output_schema = projection_mapping.output_schema(); + + let expected = expected + .into_iter() + .map(|ordering| { + ordering + .into_iter() + .map(|(name, options)| { + (col(name, &output_schema).unwrap(), options) + }) + .collect::>() + }) + .collect::>(); + let expected = convert_to_orderings_owned(&expected); + + let projected_eq = eq_properties.project(&projection_mapping, output_schema); + let orderings = projected_eq.oeq_class(); + + let err_msg = format!( + "actual: {:?}, expected: {:?}, projection_mapping: {:?}", + orderings.orderings, expected, projection_mapping + ); + + assert_eq!(orderings.len(), expected.len(), "{}", err_msg); + for expected_ordering in &expected { + assert!(orderings.contains(expected_ordering), "{}", err_msg) + } + } + + Ok(()) + } + + #[test] + fn test_expr_consists_of_ignored_exprs() -> Result<()> { let schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::Int32, true), Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + Field::new("ts", DataType::Timestamp(TimeUnit::Nanosecond, None), true), ])); - let eq_properties = EquivalenceProperties::new(schema.clone()); - // c as c_new, b as b_new - let proj_exprs = vec![ - (col("c", &schema)?, "c_new".to_string()), - (col("b", &schema)?, "b_new".to_string()), - ]; - let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &schema)?; - let projected = eq_properties - .project(&projection_mapping, projection_mapping.output_schema()); - // After projection there is no ordering. - assert!(projected.oeq_class().output_ordering().is_none()); - + let col_a = col("a", &schema)?; + let col_b = col("b", &schema)?; + let col_d = col("d", &schema)?; + let b_plus_d = Arc::new(BinaryExpr::new( + col_b.clone(), + Operator::Plus, + col_d.clone(), + )) as Arc; + + let ignored_exprs = vec![col_a.clone(), col_b.clone()]; + let expr = b_plus_d.clone(); + assert!(!expr_consists_of_ignored_exprs(&ignored_exprs, &expr)); + + let ignored_exprs = vec![col_a.clone(), col_b.clone(), col_d.clone()]; + let expr = b_plus_d.clone(); + assert!(expr_consists_of_ignored_exprs(&ignored_exprs, &expr)); Ok(()) } } diff --git a/datafusion/physical-expr/src/sort_properties.rs b/datafusion/physical-expr/src/sort_properties.rs index a3b201f84e9d..d64860c18c5b 100644 --- a/datafusion/physical-expr/src/sort_properties.rs +++ b/datafusion/physical-expr/src/sort_properties.rs @@ -17,12 +17,14 @@ use std::{ops::Neg, sync::Arc}; -use crate::PhysicalExpr; +use crate::{PhysicalExpr, PhysicalSortExpr}; use arrow_schema::SortOptions; use datafusion_common::tree_node::{TreeNode, VisitRecursion}; use datafusion_common::Result; +use crate::equivalence::EquivalenceGroup; +use crate::expressions::Literal; use itertools::Itertools; /// To propagate [`SortOptions`] across the [`PhysicalExpr`], it is insufficient @@ -167,6 +169,83 @@ impl ExprOrdering { } } + pub fn new_construction( + expr: Arc, + eq_group: &EquivalenceGroup, + leading_orderings: &[PhysicalSortExpr], + constant_exprs: &[Arc], + ) -> Self { + let expr = eq_group.normalize_expr(expr); + // Search expr among leading orderings. + let mut state = SortProperties::Unordered; + for sort_expr in leading_orderings { + if expr.eq(&sort_expr.expr) { + state = SortProperties::Ordered(sort_expr.options); + break; + } + } + for constant_expr in constant_exprs { + if expr.eq(constant_expr) { + state = SortProperties::Singleton; + break; + } + } + if expr.as_any().is::() { + state = SortProperties::Singleton; + } + if SortProperties::Unordered != state { + // If an ordered match or constant is found among leading orderings return early. + let size = expr.children().len(); + return Self { + expr, + state, + children_states: vec![SortProperties::Unordered; size], + }; + } + + if expr.children().is_empty() { + // If no children exists, we cannot continue search + // TODO: Search among equivalent groups also. + Self { + expr, + state, + children_states: vec![], + } + } else { + let children = expr + .children() + .into_iter() + .map(|child| { + ExprOrdering::new_construction( + child, + eq_group, + leading_orderings, + constant_exprs, + ) + }) + .collect::>(); + let children_states = children + .into_iter() + .map(|item| item.state) + .collect::>(); + let expr_ordering = ExprOrdering::new(expr); + let ordering = expr_ordering.expr.get_ordering(&children_states); + let mut expr_ordering = expr_ordering.with_new_children(children_states); + expr_ordering.state = ordering; + expr_ordering + } + } + + pub fn leaf_orderings( + &self, + leading_orderings: &[PhysicalSortExpr], + ) -> Option> { + let mut leaf_orderings = vec![]; + let completed = + leaf_orderings_helper(&self.expr, &mut leaf_orderings, leading_orderings); + completed.then_some(leaf_orderings) + } + /// Updates this [`ExprOrdering`]'s children states with the given states. pub fn with_new_children(mut self, children_states: Vec) -> Self { self.children_states = children_states; @@ -220,3 +299,36 @@ impl TreeNode for ExprOrdering { } } } + +fn leaf_orderings_helper( + expr: &Arc, + leaf_orderings: &mut Vec, + leading_orderings: &[PhysicalSortExpr], +) -> bool { + let children = expr.children(); + let mut state = SortProperties::Unordered; + for sort_expr in leading_orderings { + if expr.eq(&sort_expr.expr) { + state = SortProperties::Ordered(sort_expr.options); + break; + } + } + if let SortProperties::Ordered(options) = state { + let sort_expr = PhysicalSortExpr { + expr: expr.clone(), + options, + }; + leaf_orderings.push(sort_expr); + return true; + } + if expr.as_any().is::() { + return true; + } + if children.is_empty() { + false + } else { + children + .into_iter() + .all(|child| leaf_orderings_helper(&child, leaf_orderings, leading_orderings)) + } +} diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 8dab38bc5f0e..7fbf8e634236 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -613,6 +613,8 @@ impl AggregateExec { partition: usize, context: Arc, ) -> Result { + // println!("aggregate self.input.equivalence_properties():{:?}", self.input.equivalence_properties()); + // println!("aggregate self.equivalence_properties():{:?}", self.equivalence_properties()); // no group by at all if self.group_by.expr.is_empty() { return Ok(StreamType::AggregateStream(AggregateStream::new( diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 7c6794e91e48..17f8983eb052 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -209,6 +209,8 @@ 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.equivalence_properties():{:?}", self.input.equivalence_properties()); + // println!("proj self.equivalence_properties():{:?}", self.equivalence_properties()); 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/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index b6ed6e482ff5..1464561eb083 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -494,6 +494,10 @@ pub fn get_window_mode( { let req = [partition_by_reqs.clone(), order_by_reqs].concat(); let req = collapse_lex_req(req); + // println!("req : {:?}", req); + // for ordering in partition_by_eqs.oeq_class().iter(){ + // println!("ordering: {:?}", ordering); + // } if partition_by_eqs.ordering_satisfy_requirement(&req) { // Window can be run with existing ordering let mode = if indices.len() == partitionby_exprs.len() { From 49b15045583fafb2d4c6b35e5870eb66dff72445 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 10 Nov 2023 22:34:54 +0300 Subject: [PATCH 06/44] Fix failing test --- datafusion/physical-expr/src/equivalence.rs | 74 ++++++++++++--------- datafusion/physical-plan/src/windows/mod.rs | 5 +- 2 files changed, 45 insertions(+), 34 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 82f106c9eb97..c8ef850ab808 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -19,7 +19,7 @@ use std::collections::HashMap; use std::hash::Hash; use std::sync::Arc; -use crate::expressions::Column; +use crate::expressions::{Column, Literal}; use crate::physical_expr::{deduplicate_physical_exprs, have_common_entries}; use crate::sort_properties::{ExprOrdering, SortProperties}; use crate::{ @@ -1448,28 +1448,37 @@ impl EquivalenceProperties { } } } - let leading_orderings = self.get_leading_orderings(&[]); - let normalized_leading_orderings = - self.eq_group.normalize_sort_exprs(&leading_orderings); - let projected_leading_orderings = self - .project_ordering_helper(mapping, &normalized_leading_orderings) - .unwrap_or_default(); - let normalized_leading_exprs = projected_leading_orderings - .into_iter() - .map(|sort_expr| sort_expr.expr) - .collect::>(); + // let leading_orderings = self.get_leading_orderings(&[]); let leading_ordering_exprs = orderings .iter() .flat_map(|ordering| ordering.first().map(|sort_expr| sort_expr.expr.clone())) .collect::>(); + // let normalized_leading_exprs = self + // .eq_group + // .normalize_exprs(leading_ordering_exprs.clone()); + // let normalized_leading_exprs = normalized_leading_exprs + // .into_iter() + // .flat_map(|expr| self.eq_group.project_expr(mapping, &expr)) + // .collect::>(); + // let projected_leading_orderings = self + // .project_ordering_helper(mapping, &normalized_leading_orderings) + // .unwrap_or_default(); + // let normalized_leading_exprs = projected_leading_orderings + // .into_iter() + // .map(|sort_expr| sort_expr.expr) + // .collect::>(); + // let leading_ordering_exprs = orderings + // .iter() + // .flat_map(|ordering| ordering.first().map(|sort_expr| sort_expr.expr.clone())) + // .collect::>(); if PRINT_ON { - println!("normalized_leading_exprs: {:?}", normalized_leading_exprs); - println!( - "normalized_leading_orderings: {:?}", - normalized_leading_orderings - ); - println!("leading_ordering_exprs: {:?}", leading_ordering_exprs); + // println!("leading_ordering_exprs: {:?}", leading_ordering_exprs); + // println!("normalized_leading_exprs: {:?}", normalized_leading_exprs); + // println!( + // "normalized_leading_orderings: {:?}", + // normalized_leading_orderings + // ); } orderings = orderings .into_iter() @@ -1498,7 +1507,9 @@ impl EquivalenceProperties { }) .collect(); if PRINT_ON { + println!("mapping:{:?}", mapping); println!("leading_ordering_exprs: {:?}", leading_ordering_exprs); + // println!("normalized_leading_exprs: {:?}", normalized_leading_exprs); println!("self.constants: {:?}", self.constants); println!("self.eq_group: {:?}", self.eq_group); for ordering in &orderings { @@ -1776,28 +1787,27 @@ fn update_ordering( mut node: ExprOrdering, eq_properties: &EquivalenceProperties, ) -> Result> { + // We have a Column, which is one of the two possible leaf node types: + let eq_group = &eq_properties.eq_group; + let normalized_expr = eq_group.normalize_expr(node.expr.clone()); + let oeq_class = &eq_properties.normalized_oeq_class(); + if eq_properties.is_expr_constant(&normalized_expr) { + node.state = SortProperties::Singleton; + return Ok(Transformed::Yes(node)); + } else if let Some(options) = oeq_class.get_options(&normalized_expr) { + node.state = SortProperties::Ordered(options); + return Ok(Transformed::Yes(node)); + } if !node.expr.children().is_empty() { // We have an intermediate (non-leaf) node, account for its children: node.state = node.expr.get_ordering(&node.children_states); 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 = &eq_properties.eq_group; - let normalized_expr = eq_group.normalize_expr(node.expr.clone()); - let oeq_class = &eq_properties.normalized_oeq_class(); - if eq_properties.is_expr_constant(&normalized_expr) { - node.state = SortProperties::Singleton; - Ok(Transformed::Yes(node)) - } else if let Some(options) = oeq_class.get_options(&normalized_expr) { - node.state = SortProperties::Ordered(options); - Ok(Transformed::Yes(node)) - } else { - Ok(Transformed::No(node)) - } - } else { + } else if node.expr.as_any().is::() { // We have a Literal, which is the other possible leaf node type: node.state = node.expr.get_ordering(&[]); Ok(Transformed::Yes(node)) + } else { + Ok(Transformed::No(node)) } } diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 1464561eb083..9d2b1432c76f 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -494,11 +494,12 @@ pub fn get_window_mode( { let req = [partition_by_reqs.clone(), order_by_reqs].concat(); let req = collapse_lex_req(req); - // println!("req : {:?}", req); + // println!("window req : {:?}", req); // for ordering in partition_by_eqs.oeq_class().iter(){ - // println!("ordering: {:?}", ordering); + // println!("window ordering: {:?}", ordering); // } if partition_by_eqs.ordering_satisfy_requirement(&req) { + // println!("satisfied"); // Window can be run with existing ordering let mode = if indices.len() == partitionby_exprs.len() { PartitionSearchMode::Sorted From 4ce3b9077df0c68c62a3638490e2c4e61b6a1f38 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 10 Nov 2023 22:55:22 +0300 Subject: [PATCH 07/44] Simplifications --- datafusion/physical-expr/src/equivalence.rs | 62 +---------------- .../physical-expr/src/sort_properties.rs | 69 ------------------- 2 files changed, 3 insertions(+), 128 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index c8ef850ab808..61745f88cfaa 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1249,17 +1249,10 @@ impl EquivalenceProperties { &self, mapping: &ProjectionMapping, ignored_exprs: &[Arc], - constants: &[Arc], ) -> Vec> { let mut new_orderings = vec![]; let leading_orderings = self.get_leading_orderings(ignored_exprs); for (source, target) in mapping.iter() { - // let expr_ordering = ExprOrdering::new_construction( - // source.clone(), - // &self.eq_group, - // &leading_orderings, - // constants, - // ); let expr_ordering = self.get_expr_ordering(source.clone()); if PRINT_ON { println!("expr_ordering: {expr_ordering:?}"); @@ -1360,10 +1353,8 @@ impl EquivalenceProperties { // valid after projection. let mut orderings = vec![vec![]]; let mut ignored_exprs = vec![]; - let mut constants = self.constants.clone(); for sort_expr in ordering.iter() { - let new_orderings = - self.calc_ordered_exprs(mapping, &ignored_exprs, &constants); + let new_orderings = self.calc_ordered_exprs(mapping, &ignored_exprs); let mut projected_ignored_exprs = if let Some(ignored_exprs) = ignored_exprs .iter() .map(|expr| self.eq_group.project_expr(mapping, expr)) @@ -1428,7 +1419,6 @@ impl EquivalenceProperties { let leaves = self.eq_group.normalize_exprs(leaves); ignored_exprs.extend(leaves); ignored_exprs.push(sort_expr.expr.clone()); - constants.push(sort_expr.expr.clone()); self = self.add_constants(std::iter::once(sort_expr.expr.clone())); deduplicate_physical_exprs(&mut ignored_exprs); if PRINT_ON { @@ -1453,63 +1443,17 @@ impl EquivalenceProperties { .iter() .flat_map(|ordering| ordering.first().map(|sort_expr| sort_expr.expr.clone())) .collect::>(); - // let normalized_leading_exprs = self - // .eq_group - // .normalize_exprs(leading_ordering_exprs.clone()); - // let normalized_leading_exprs = normalized_leading_exprs - // .into_iter() - // .flat_map(|expr| self.eq_group.project_expr(mapping, &expr)) - // .collect::>(); - // let projected_leading_orderings = self - // .project_ordering_helper(mapping, &normalized_leading_orderings) - // .unwrap_or_default(); - // let normalized_leading_exprs = projected_leading_orderings - // .into_iter() - // .map(|sort_expr| sort_expr.expr) - // .collect::>(); - // let leading_ordering_exprs = orderings - // .iter() - // .flat_map(|ordering| ordering.first().map(|sort_expr| sort_expr.expr.clone())) - // .collect::>(); - if PRINT_ON { - // println!("leading_ordering_exprs: {:?}", leading_ordering_exprs); - // println!("normalized_leading_exprs: {:?}", normalized_leading_exprs); - // println!( - // "normalized_leading_orderings: {:?}", - // normalized_leading_orderings - // ); - } orderings = orderings .into_iter() .map(|ordering| { let collapsed = collapse_lex_ordering(ordering, &leading_ordering_exprs); - let normalized = self.eq_group.normalize_sort_exprs(&collapsed); - // println!("normalized: {:?}", normalized); - // println!("normalized_leading_exprs: {:?}", normalized_leading_exprs); - // println!("leading_ordering_exprs: {:?}", leading_ordering_exprs); - normalized - .into_iter() - .enumerate() - .flat_map(|(idx, sort_expr)| { - if idx > 0 - && physical_exprs_contains( - &leading_ordering_exprs, - &sort_expr.expr, - ) - { - None - } else { - Some(sort_expr) - } - }) - .collect::>() + self.eq_group.normalize_sort_exprs(&collapsed) }) .collect(); if PRINT_ON { println!("mapping:{:?}", mapping); println!("leading_ordering_exprs: {:?}", leading_ordering_exprs); - // println!("normalized_leading_exprs: {:?}", normalized_leading_exprs); println!("self.constants: {:?}", self.constants); println!("self.eq_group: {:?}", self.eq_group); for ordering in &orderings { @@ -1523,7 +1467,7 @@ impl EquivalenceProperties { /// Projects the equivalences within according to `projection_mapping` /// and `output_schema`. pub fn project( - mut self, + &self, projection_mapping: &ProjectionMapping, output_schema: SchemaRef, ) -> Self { diff --git a/datafusion/physical-expr/src/sort_properties.rs b/datafusion/physical-expr/src/sort_properties.rs index d64860c18c5b..905c831908a3 100644 --- a/datafusion/physical-expr/src/sort_properties.rs +++ b/datafusion/physical-expr/src/sort_properties.rs @@ -22,8 +22,6 @@ use crate::{PhysicalExpr, PhysicalSortExpr}; use arrow_schema::SortOptions; use datafusion_common::tree_node::{TreeNode, VisitRecursion}; use datafusion_common::Result; - -use crate::equivalence::EquivalenceGroup; use crate::expressions::Literal; use itertools::Itertools; @@ -169,73 +167,6 @@ impl ExprOrdering { } } - pub fn new_construction( - expr: Arc, - eq_group: &EquivalenceGroup, - leading_orderings: &[PhysicalSortExpr], - constant_exprs: &[Arc], - ) -> Self { - let expr = eq_group.normalize_expr(expr); - // Search expr among leading orderings. - let mut state = SortProperties::Unordered; - for sort_expr in leading_orderings { - if expr.eq(&sort_expr.expr) { - state = SortProperties::Ordered(sort_expr.options); - break; - } - } - for constant_expr in constant_exprs { - if expr.eq(constant_expr) { - state = SortProperties::Singleton; - break; - } - } - if expr.as_any().is::() { - state = SortProperties::Singleton; - } - if SortProperties::Unordered != state { - // If an ordered match or constant is found among leading orderings return early. - let size = expr.children().len(); - return Self { - expr, - state, - children_states: vec![SortProperties::Unordered; size], - }; - } - - if expr.children().is_empty() { - // If no children exists, we cannot continue search - // TODO: Search among equivalent groups also. - Self { - expr, - state, - children_states: vec![], - } - } else { - let children = expr - .children() - .into_iter() - .map(|child| { - ExprOrdering::new_construction( - child, - eq_group, - leading_orderings, - constant_exprs, - ) - }) - .collect::>(); - let children_states = children - .into_iter() - .map(|item| item.state) - .collect::>(); - let expr_ordering = ExprOrdering::new(expr); - let ordering = expr_ordering.expr.get_ordering(&children_states); - let mut expr_ordering = expr_ordering.with_new_children(children_states); - expr_ordering.state = ordering; - expr_ordering - } - } - pub fn leaf_orderings( &self, leading_orderings: &[PhysicalSortExpr], From 78206ba4672492c02d207d1e59d3137a2f8f5a68 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 10 Nov 2023 23:11:41 +0300 Subject: [PATCH 08/44] Simplifications --- datafusion/physical-expr/src/equivalence.rs | 52 +++++-------------- .../physical-expr/src/sort_properties.rs | 2 +- 2 files changed, 13 insertions(+), 41 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 61745f88cfaa..e2ab235030b2 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1248,10 +1248,9 @@ impl EquivalenceProperties { fn calc_ordered_exprs( &self, mapping: &ProjectionMapping, - ignored_exprs: &[Arc], ) -> Vec> { let mut new_orderings = vec![]; - let leading_orderings = self.get_leading_orderings(ignored_exprs); + let leading_orderings = self.get_leading_orderings(&self.constants); for (source, target) in mapping.iter() { let expr_ordering = self.get_expr_ordering(source.clone()); if PRINT_ON { @@ -1290,7 +1289,7 @@ impl EquivalenceProperties { let suffix = suffix.into_iter().cloned(); new_ordering.extend(suffix); let new_ordering = - collapse_lex_ordering(new_ordering, ignored_exprs); + collapse_lex_ordering(new_ordering, &self.constants); if !new_orderings.contains(&new_ordering) { new_orderings.push(new_ordering); } @@ -1315,7 +1314,7 @@ impl EquivalenceProperties { // TODO: send stop signal. } } - let new_ordering = collapse_lex_ordering(new_ordering, ignored_exprs); + let new_ordering = collapse_lex_ordering(new_ordering, &self.constants); if !new_orderings.contains(&new_ordering) { new_orderings.push(new_ordering); } @@ -1352,25 +1351,25 @@ impl EquivalenceProperties { // the result should be [a ASC], not [a ASC, c ASC], even if column c is // valid after projection. let mut orderings = vec![vec![]]; - let mut ignored_exprs = vec![]; for sort_expr in ordering.iter() { - let new_orderings = self.calc_ordered_exprs(mapping, &ignored_exprs); - let mut projected_ignored_exprs = if let Some(ignored_exprs) = ignored_exprs + let new_orderings = self.calc_ordered_exprs(mapping); + let mut projected_constants = if let Some(projected_constants) = self + .constants .iter() .map(|expr| self.eq_group.project_expr(mapping, expr)) .collect::>>() { - ignored_exprs + projected_constants } else { - // At least one of the ignored_exprs cannot be projected. Hence continuing the iteration is not guaranteed to consider all referred columns. + // At least one of the constants cannot be projected. Hence continuing the iteration is not guaranteed to consider all referred columns. // Stop iteration. break; }; for (source, target) in mapping.iter() { - if expr_consists_of_ignored_exprs(&ignored_exprs, source) - && !physical_exprs_contains(&projected_ignored_exprs, source) + if expr_consists_of_ignored_exprs(&self.constants, source) + && !physical_exprs_contains(&projected_constants, source) { - projected_ignored_exprs.push(target.clone()); + projected_constants.push(target.clone()); } } @@ -1388,7 +1387,7 @@ impl EquivalenceProperties { .enumerate() .flat_map(|(idx, ordering)| { if ordering_consists_of_ignored_exprs( - &projected_ignored_exprs, + &projected_constants, ordering, ) { Some(idx) @@ -1415,12 +1414,7 @@ impl EquivalenceProperties { orderings[ordering_idx].extend(new_ordering.to_vec()); } } - let leaves = leaf_exprs(sort_expr.expr.clone()); - let leaves = self.eq_group.normalize_exprs(leaves); - ignored_exprs.extend(leaves); - ignored_exprs.push(sort_expr.expr.clone()); self = self.add_constants(std::iter::once(sort_expr.expr.clone())); - deduplicate_physical_exprs(&mut ignored_exprs); if PRINT_ON { for ordering in &orderings { println!("updated ordering: {ordering:?}"); @@ -1438,7 +1432,6 @@ impl EquivalenceProperties { } } } - // let leading_orderings = self.get_leading_orderings(&[]); let leading_ordering_exprs = orderings .iter() .flat_map(|ordering| ordering.first().map(|sort_expr| sort_expr.expr.clone())) @@ -1598,27 +1591,6 @@ fn ordering_consists_of_ignored_exprs( .all(|sort_expr| expr_consists_of_ignored_exprs(ignored_exprs, &sort_expr.expr)) } -fn leaf_exprs(expr: Arc) -> Vec> { - let mut result = vec![]; - leaf_exprs_helper(expr, &mut result); - result -} - -fn leaf_exprs_helper( - expr: Arc, - result: &mut Vec>, -) { - if expr.children().is_empty() { - if !physical_exprs_contains(result, &expr) { - result.push(expr); - } - } else { - expr.children() - .into_iter() - .for_each(|child| leaf_exprs_helper(child, result)); - } -} - /// Calculate ordering equivalence properties for the given join operation. pub fn join_equivalence_properties( left: EquivalenceProperties, diff --git a/datafusion/physical-expr/src/sort_properties.rs b/datafusion/physical-expr/src/sort_properties.rs index 905c831908a3..1845226b0ae7 100644 --- a/datafusion/physical-expr/src/sort_properties.rs +++ b/datafusion/physical-expr/src/sort_properties.rs @@ -19,10 +19,10 @@ use std::{ops::Neg, sync::Arc}; use crate::{PhysicalExpr, PhysicalSortExpr}; +use crate::expressions::Literal; use arrow_schema::SortOptions; use datafusion_common::tree_node::{TreeNode, VisitRecursion}; use datafusion_common::Result; -use crate::expressions::Literal; use itertools::Itertools; /// To propagate [`SortOptions`] across the [`PhysicalExpr`], it is insufficient From 429096a8f6b98dedfdfb0f72a5e914142372e070 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Sat, 11 Nov 2023 00:15:07 +0300 Subject: [PATCH 09/44] Add is end flag --- datafusion/physical-expr/src/equivalence.rs | 72 ++++++++++++++++----- 1 file changed, 55 insertions(+), 17 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index e2ab235030b2..65123e670c74 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1248,7 +1248,7 @@ impl EquivalenceProperties { fn calc_ordered_exprs( &self, mapping: &ProjectionMapping, - ) -> Vec> { + ) -> Vec<(Vec, bool)> { let mut new_orderings = vec![]; let leading_orderings = self.get_leading_orderings(&self.constants); for (source, target) in mapping.iter() { @@ -1272,6 +1272,7 @@ impl EquivalenceProperties { options, }; let new_ordering = vec![sort_expr.clone()]; + let mut is_end = false; if let Some(leaf_orderings) = expr_ordering.leaf_orderings(&leading_orderings) { @@ -1290,12 +1291,14 @@ impl EquivalenceProperties { new_ordering.extend(suffix); let new_ordering = collapse_lex_ordering(new_ordering, &self.constants); - if !new_orderings.contains(&new_ordering) { - new_orderings.push(new_ordering); + if !new_orderings + .contains(&(new_ordering.clone(), is_end)) + { + new_orderings.push((new_ordering, is_end)); } } - // new_ordering.extend(projected_leaf_orderings); } else { + is_end = true; if PRINT_ON { println!( "send stop signal, expr_ordering: {:?}", @@ -1305,6 +1308,7 @@ impl EquivalenceProperties { // TODO: send stop signal. } } else { + is_end = true; if PRINT_ON { println!( "send stop signal, expr_ordering: {:?}", @@ -1315,8 +1319,8 @@ impl EquivalenceProperties { } } let new_ordering = collapse_lex_ordering(new_ordering, &self.constants); - if !new_orderings.contains(&new_ordering) { - new_orderings.push(new_ordering); + if !new_orderings.contains(&(new_ordering.clone(), is_end)) { + new_orderings.push((new_ordering, is_end)); } } } @@ -1324,8 +1328,8 @@ impl EquivalenceProperties { if let Some(projected_leading_ordering) = self.project_ordering_helper(mapping, &[leading_ordering]) { - if !new_orderings.contains(&projected_leading_ordering) { - new_orderings.push(projected_leading_ordering); + if !new_orderings.contains(&(projected_leading_ordering.clone(), false)) { + new_orderings.push((projected_leading_ordering, false)); } } } @@ -1334,7 +1338,7 @@ impl EquivalenceProperties { println!("new_ordering: {new_ordering:?}"); } } - new_orderings.retain(|ordering| !ordering.is_empty()); + new_orderings.retain(|(ordering, _is_end)| !ordering.is_empty()); new_orderings } @@ -1350,7 +1354,8 @@ impl EquivalenceProperties { // 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 mut orderings = vec![vec![]]; + let mut orderings: Vec<(LexOrdering, bool)> = vec![(vec![], false)]; + let mut completed_orderings_holder: Vec = vec![]; for sort_expr in ordering.iter() { let new_orderings = self.calc_ordered_exprs(mapping); let mut projected_constants = if let Some(projected_constants) = self @@ -1385,11 +1390,12 @@ impl EquivalenceProperties { let relevant_indices = orderings .iter() .enumerate() - .flat_map(|(idx, ordering)| { + .flat_map(|(idx, (ordering, is_end))| { if ordering_consists_of_ignored_exprs( &projected_constants, ordering, - ) { + ) && !*is_end + { Some(idx) } else { None @@ -1410,8 +1416,9 @@ impl EquivalenceProperties { for (iter_idx, ordering_idx) in new_indices.into_iter().enumerate() { let mod_idx = iter_idx % new_orderings.len(); - let new_ordering = &new_orderings[mod_idx]; - orderings[ordering_idx].extend(new_ordering.to_vec()); + let (new_ordering, is_end) = &new_orderings[mod_idx]; + orderings[ordering_idx].0.extend(new_ordering.to_vec()); + orderings[ordering_idx].1 = *is_end; } } self = self.add_constants(std::iter::once(sort_expr.expr.clone())); @@ -1420,18 +1427,49 @@ impl EquivalenceProperties { println!("updated ordering: {ordering:?}"); } } - orderings = orderings + if PRINT_ON { + println!("orderings:{:?}", orderings); + } + let (completed_orderings, continuing_orderings): (Vec<_>, Vec<_>) = orderings + .into_iter() + .partition(|(_ordering, is_end)| *is_end); + let continuing_orderings = continuing_orderings + .into_iter() + .map(|(ordering, _is_end)| ordering) + .collect::>(); + let completed_orderings = completed_orderings + .into_iter() + .map(|(ordering, _is_end)| ordering) + .collect::>(); + if PRINT_ON { + println!("continuing_orderings:{:?}", continuing_orderings); + println!("completed_orderings:{:?}", completed_orderings); + } + completed_orderings_holder.extend(completed_orderings); + let continuing_orderings = continuing_orderings .into_iter() .map(|ordering| self.eq_group.normalize_sort_exprs(&ordering)) .collect(); - let oeq_class = OrderingEquivalenceClass::new(orderings); - orderings = oeq_class.orderings; + let oeq_class = OrderingEquivalenceClass::new(continuing_orderings); + orderings = oeq_class + .orderings + .into_iter() + .map(|ordering| (ordering, false)) + .collect::<_>(); if PRINT_ON { for ordering in &orderings { println!("updated ordering2: {ordering:?}"); } } } + let mut orderings = orderings + .into_iter() + .map(|(ordering, _is_end)| ordering) + .collect::>(); + orderings.extend(completed_orderings_holder); + let oeq_class = OrderingEquivalenceClass::new(orderings); + orderings = oeq_class.orderings; + let leading_ordering_exprs = orderings .iter() .flat_map(|ordering| ordering.first().map(|sort_expr| sort_expr.expr.clone())) From a4a3d7e38cdebb96e4dd5ce77ac22224056736bf Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 14 Nov 2023 16:26:25 +0300 Subject: [PATCH 10/44] Simplifications --- .../src/physical_optimizer/enforce_sorting.rs | 217 ---------------- .../replace_with_order_preserving_variants.rs | 244 ++++++++++-------- .../physical-plan/src/aggregates/mod.rs | 2 - datafusion/physical-plan/src/projection.rs | 2 - datafusion/physical-plan/src/windows/mod.rs | 5 - 5 files changed, 131 insertions(+), 339 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 3f0ae4400f5a..4779ced44f1a 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -2278,220 +2278,3 @@ mod tests { Ok(()) } } - -#[cfg(test)] -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_order() -> Result<()> { - let config = SessionConfig::new().with_target_partitions(1); - let ctx = SessionContext::new_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 - 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 batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; - print_batches(&batches)?; - - let expected = vec![ - "GlobalLimitExec: skip=0, fetch=5", - " 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, actual, - "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - ); - - let expected = [ - "+--------+", - "| result |", - "+--------+", - "| 0 |", - "| 1 |", - "| 2 |", - "| 3 |", - "| 4 |", - "+--------+", - ]; - assert_batches_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn test_window() -> Result<()> { - let config = SessionConfig::new().with_target_partitions(1); - let ctx = SessionContext::new_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 batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; - print_batches(&batches)?; - - let expected = 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, actual, - "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - ); - - let expected = [ - "+-----+-------------+--------------+", - "| c3 | sum1 | sum2 |", - "+-----+-------------+--------------+", - "| -86 | 2861911482 | 222089770060 |", - "| 13 | 5075947208 | 219227858578 |", - "| 125 | 8701233618 | 217013822852 |", - "| 123 | 11293564174 | 213388536442 |", - "| 97 | 14767488750 | 210796205886 |", - "+-----+-------------+--------------+", - ]; - assert_batches_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn test_aggregate() -> Result<()> { - let config = SessionConfig::new().with_target_partitions(1); - let ctx = SessionContext::new_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 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 - 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 batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; - print_batches(&batches)?; - - let expected = vec![ - "GlobalLimitExec: skip=0, fetch=5", - " 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=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, 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, 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 expected = [ - "+---+-----------+", - "| a | last_col1 |", - "+---+-----------+", - "| 0 | 1 |", - "| 0 | 1 |", - "| 0 | 1 |", - "| 0 | 1 |", - "| 0 | 1 |", - "+---+-----------+", - ]; - assert_batches_eq!(expected, &batches); - 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 cb7067263e1b..69c1f40d9cdd 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 @@ -358,13 +358,13 @@ mod tests { let expected_input = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortExec: expr=[a@0 ASC NULLS LAST]", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true",]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", + " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true",]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) } @@ -378,40 +378,43 @@ mod tests { let repartition_hash = repartition_exec_hash(repartition_rr); let coalesce_partitions = coalesce_partitions_exec(repartition_hash); let sort = sort_exec( - vec![sort_expr_default("a", &schema)], + vec![sort_expr_default("a", &coalesce_partitions.schema())], coalesce_partitions, false, ); let repartition_rr2 = repartition_exec_round_robin(sort); let repartition_hash2 = repartition_exec_hash(repartition_rr2); - let filter = filter_exec(repartition_hash2, &schema); - let sort2 = sort_exec(vec![sort_expr_default("a", &schema)], filter, true); + let filter = filter_exec(repartition_hash2); + let sort2 = + sort_exec(vec![sort_expr_default("a", &filter.schema())], filter, true); - let physical_plan = - sort_preserving_merge_exec(vec![sort_expr_default("a", &schema)], sort2); + let physical_plan = sort_preserving_merge_exec( + vec![sort_expr_default("a", &sort2.schema())], + sort2, + ); let expected_input = [ "SortPreservingMergeExec: [a@0 ASC]", " SortExec: expr=[a@0 ASC]", - " FilterExec: c@2 > 3", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " SortExec: expr=[a@0 ASC]", " CoalescePartitionsExec", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC], has_header=true", ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC]", - " FilterExec: c@2 > 3", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC", + " FilterExec: c@1 > 3", + " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " SortPreservingMergeExec: [a@0 ASC]", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC", + " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC], has_header=true", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) @@ -423,7 +426,7 @@ mod tests { let sort_exprs = vec![sort_expr("a", &schema)]; let source = csv_exec_sorted(&schema, sort_exprs, true); let repartition_rr = repartition_exec_round_robin(source); - let filter = filter_exec(repartition_rr, &schema); + let filter = filter_exec(repartition_rr); let repartition_hash = repartition_exec_hash(filter); let sort = sort_exec(vec![sort_expr("a", &schema)], repartition_hash, true); @@ -432,16 +435,16 @@ mod tests { let expected_input = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortExec: expr=[a@0 ASC NULLS LAST]", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", - " FilterExec: c@2 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " FilterExec: c@1 > 3", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", - " FilterExec: c@2 > 3", + " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", + " FilterExec: c@1 > 3", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); @@ -455,7 +458,7 @@ mod tests { let source = csv_exec_sorted(&schema, sort_exprs, true); let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); - let filter = filter_exec(repartition_hash, &schema); + let filter = filter_exec(repartition_hash); let coalesce_batches_exec: Arc = coalesce_batches_exec(filter); let sort = sort_exec(vec![sort_expr("a", &schema)], coalesce_batches_exec, true); @@ -465,16 +468,16 @@ mod tests { let expected_input = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortExec: expr=[a@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@2 > 3", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@2 > 3", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", + " FilterExec: c@1 > 3", + " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) } @@ -487,7 +490,7 @@ mod tests { let repartition_rr = repartition_exec_round_robin(source); let coalesce_batches_exec_1 = coalesce_batches_exec(repartition_rr); let repartition_hash = repartition_exec_hash(coalesce_batches_exec_1); - let filter = filter_exec(repartition_hash, &schema); + let filter = filter_exec(repartition_hash); let coalesce_batches_exec_2 = coalesce_batches_exec(filter); let sort = sort_exec(vec![sort_expr("a", &schema)], coalesce_batches_exec_2, true); @@ -498,18 +501,18 @@ mod tests { let expected_input = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortExec: expr=[a@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@2 > 3", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@2 > 3", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", + " FilterExec: c@1 > 3", + " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) } @@ -521,7 +524,7 @@ mod tests { let source = csv_exec_sorted(&schema, sort_exprs, true); let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); - let filter = filter_exec(repartition_hash, &schema); + let filter = filter_exec(repartition_hash); let coalesce_batches_exec: Arc = coalesce_batches_exec(filter); let physical_plan: Arc = @@ -529,16 +532,16 @@ mod tests { let expected_input = ["CoalescePartitionsExec", " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@2 > 3", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; let expected_optimized = ["CoalescePartitionsExec", " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@2 > 3", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) } @@ -550,7 +553,7 @@ mod tests { let source = csv_exec_sorted(&schema, sort_exprs, true); let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); - let filter = filter_exec(repartition_hash, &schema); + let filter = filter_exec(repartition_hash); let coalesce_batches = coalesce_batches_exec(filter); let repartition_hash_2 = repartition_exec_hash(coalesce_batches); let sort = sort_exec(vec![sort_expr("a", &schema)], repartition_hash_2, true); @@ -561,21 +564,21 @@ mod tests { let expected_input = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortExec: expr=[a@0 ASC NULLS LAST]", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@2 > 3", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true" + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true" ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", + " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@2 > 3", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", + " FilterExec: c@1 > 3", + " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) @@ -589,24 +592,26 @@ mod tests { let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let sort = sort_exec( - vec![sort_expr_default("c", &schema)], + vec![sort_expr_default("c", &repartition_hash.schema())], repartition_hash, true, ); - let physical_plan = - sort_preserving_merge_exec(vec![sort_expr_default("c", &schema)], sort); + let physical_plan = sort_preserving_merge_exec( + vec![sort_expr_default("c", &sort.schema())], + sort, + ); - let expected_input = ["SortPreservingMergeExec: [c@2 ASC]", - " SortExec: expr=[c@2 ASC]", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + let expected_input = ["SortPreservingMergeExec: [c@1 ASC]", + " SortExec: expr=[c@1 ASC]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; - let expected_optimized = ["SortPreservingMergeExec: [c@2 ASC]", - " SortExec: expr=[c@2 ASC]", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true",]; + let expected_optimized = ["SortPreservingMergeExec: [c@1 ASC]", + " SortExec: expr=[c@1 ASC]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true",]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) } @@ -624,13 +629,13 @@ mod tests { let expected_input = ["SortExec: expr=[a@0 ASC NULLS LAST]", " CoalescePartitionsExec", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", + " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) } @@ -644,41 +649,44 @@ mod tests { let repartition_hash = repartition_exec_hash(repartition_rr); let coalesce_partitions = coalesce_partitions_exec(repartition_hash); let sort = sort_exec( - vec![sort_expr_default("c", &schema)], + vec![sort_expr_default("c", &coalesce_partitions.schema())], coalesce_partitions, false, ); let repartition_rr2 = repartition_exec_round_robin(sort); let repartition_hash2 = repartition_exec_hash(repartition_rr2); - let filter = filter_exec(repartition_hash2, &schema); - let sort2 = sort_exec(vec![sort_expr_default("c", &schema)], filter, true); + let filter = filter_exec(repartition_hash2); + let sort2 = + sort_exec(vec![sort_expr_default("c", &filter.schema())], filter, true); - let physical_plan = - sort_preserving_merge_exec(vec![sort_expr_default("c", &schema)], sort2); + let physical_plan = sort_preserving_merge_exec( + vec![sort_expr_default("c", &sort2.schema())], + sort2, + ); let expected_input = [ - "SortPreservingMergeExec: [c@2 ASC]", - " SortExec: expr=[c@2 ASC]", - " FilterExec: c@2 > 3", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + "SortPreservingMergeExec: [c@1 ASC]", + " SortExec: expr=[c@1 ASC]", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " SortExec: expr=[c@2 ASC]", + " SortExec: expr=[c@1 ASC]", " CoalescePartitionsExec", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; let expected_optimized = [ - "SortPreservingMergeExec: [c@2 ASC]", - " FilterExec: c@2 > 3", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=c@2 ASC", + "SortPreservingMergeExec: [c@1 ASC]", + " FilterExec: c@1 > 3", + " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=c@1 ASC", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " SortExec: expr=[c@2 ASC]", + " SortExec: expr=[c@1 ASC]", " CoalescePartitionsExec", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) @@ -704,37 +712,43 @@ mod tests { let hash_join_exec = hash_join_exec(left_coalesce_partitions, right_coalesce_partitions); - let sort = sort_exec(vec![sort_expr_default("a", &schema)], hash_join_exec, true); + let sort = sort_exec( + vec![sort_expr_default("a", &hash_join_exec.schema())], + hash_join_exec, + true, + ); - let physical_plan = - sort_preserving_merge_exec(vec![sort_expr_default("a", &schema)], sort); + let physical_plan = sort_preserving_merge_exec( + vec![sort_expr_default("a", &sort.schema())], + sort, + ); let expected_input = [ "SortPreservingMergeExec: [a@0 ASC]", " SortExec: expr=[a@0 ASC]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c@2)]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)]", " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC]", " SortExec: expr=[a@0 ASC]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c@2)]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)]", " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) @@ -753,13 +767,13 @@ mod tests { let expected_input = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortExec: expr=[a@0 ASC NULLS LAST]", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", + " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) } @@ -820,24 +834,23 @@ mod tests { } fn repartition_exec_hash(input: Arc) -> Arc { + let input_schema = input.schema(); Arc::new( RepartitionExec::try_new( input, - Partitioning::Hash(vec![Arc::new(Column::new("c1", 0))], 8), + Partitioning::Hash(vec![col("c", &input_schema).unwrap()], 8), ) .unwrap(), ) } - fn filter_exec( - input: Arc, - schema: &SchemaRef, - ) -> Arc { + fn filter_exec(input: Arc) -> Arc { + let input_schema = input.schema(); let predicate = expressions::binary( - col("c", schema).unwrap(), + col("c", &input_schema).unwrap(), Operator::Gt, expressions::lit(3i32), - schema, + &input_schema, ) .unwrap(); Arc::new(FilterExec::try_new(predicate, input).unwrap()) @@ -855,11 +868,15 @@ mod tests { left: Arc, right: Arc, ) -> Arc { + let left_on = col("c", &left.schema()).unwrap(); + let right_on = col("c", &right.schema()).unwrap(); + let left_col = left_on.as_any().downcast_ref::().unwrap(); + let right_col = right_on.as_any().downcast_ref::().unwrap(); Arc::new( HashJoinExec::try_new( left, right, - vec![(Column::new("c", 2), Column::new("c", 2))], + vec![(left_col.clone(), right_col.clone())], None, &JoinType::Inner, PartitionMode::Partitioned, @@ -887,6 +904,7 @@ mod tests { infinite_source: bool, ) -> Arc { let sort_exprs = sort_exprs.into_iter().collect(); + let projection: Vec = vec![0, 2, 3]; Arc::new(CsvExec::new( FileScanConfig { @@ -897,7 +915,7 @@ mod tests { 100, )]], statistics: Statistics::new_unknown(schema), - projection: None, + projection: Some(projection), limit: None, table_partition_cols: vec![], output_ordering: vec![sort_exprs], diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 7fbf8e634236..8dab38bc5f0e 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -613,8 +613,6 @@ impl AggregateExec { partition: usize, context: Arc, ) -> Result { - // println!("aggregate self.input.equivalence_properties():{:?}", self.input.equivalence_properties()); - // println!("aggregate self.equivalence_properties():{:?}", self.equivalence_properties()); // no group by at all if self.group_by.expr.is_empty() { return Ok(StreamType::AggregateStream(AggregateStream::new( diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 17f8983eb052..7c6794e91e48 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -209,8 +209,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.equivalence_properties():{:?}", self.input.equivalence_properties()); - // println!("proj self.equivalence_properties():{:?}", self.equivalence_properties()); 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/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 9d2b1432c76f..b6ed6e482ff5 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -494,12 +494,7 @@ pub fn get_window_mode( { let req = [partition_by_reqs.clone(), order_by_reqs].concat(); let req = collapse_lex_req(req); - // println!("window req : {:?}", req); - // for ordering in partition_by_eqs.oeq_class().iter(){ - // println!("window ordering: {:?}", ordering); - // } if partition_by_eqs.ordering_satisfy_requirement(&req) { - // println!("satisfied"); // Window can be run with existing ordering let mode = if indices.len() == partitionby_exprs.len() { PartitionSearchMode::Sorted From f6bc4d96160e758b32fbd056224d0bc9484ddba5 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 14 Nov 2023 16:34:34 +0300 Subject: [PATCH 11/44] Simplifications --- datafusion/physical-expr/src/equivalence.rs | 40 +++++++++++---------- 1 file changed, 21 insertions(+), 19 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 65123e670c74..64ec7d7f21b9 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -441,22 +441,24 @@ impl EquivalenceGroup { mapping: &ProjectionMapping, expr: &Arc, ) -> Option> { - for (source, target) in mapping.iter() { - // If we match the source, we can project. For example, if we have the mapping - // (a as a1, a + c) `a` projects to `a1`, and `binary_expr(a+b)` projects to `col(a+b)`. - if source.eq(expr) { - return Some(target.clone()); - } - } - for (source, target) in mapping.iter() { - // If we match 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` projects to `a1`. - if self - .get_equivalence_class(source) - .map_or(false, |group| physical_exprs_contains(group, expr)) - { - return Some(target.clone()); + // First, we try to project expressions with an exact match. If we are + // unable to do this, we consult equivalence classes. + if let Some(target) = mapping.target_expr(expr) { + // If we match the source, we can project directly: + return Some(target); + } else { + // If the given expression is not inside the mapping, try to project + // expressions considering the equivalence classes. + for (source, target) in mapping.iter() { + // If we match 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` projects to `a1`. + if self + .get_equivalence_class(source) + .map_or(false, |group| physical_exprs_contains(group, expr)) + { + return Some(target.clone()); + } } } // Project a non-leaf expression by projecting its children. @@ -1771,9 +1773,11 @@ mod tests { use std::sync::Arc; use super::*; + + use crate::execution_props::ExecutionProps; use crate::expressions::{col, lit, BinaryExpr, Column, Literal}; + use crate::functions::create_physical_expr; use crate::physical_expr::{physical_exprs_bag_equal, physical_exprs_equal}; - use arrow::compute::{lexsort_to_indices, SortColumn}; use arrow::datatypes::{DataType, Field, Schema}; use arrow_array::{ArrayRef, Float64Array, RecordBatch, UInt32Array}; @@ -1781,8 +1785,6 @@ mod tests { use datafusion_common::{Result, ScalarValue}; use datafusion_expr::{BuiltinScalarFunction, Operator}; - use crate::execution_props::ExecutionProps; - use crate::functions::create_physical_expr; use itertools::{izip, Itertools}; use rand::rngs::StdRng; use rand::seq::SliceRandom; From 20ade77ce6f176dd6a97f1cfe5cc68cce24613b8 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 14 Nov 2023 16:59:39 +0300 Subject: [PATCH 12/44] Simplifications --- datafusion/physical-expr/src/equivalence.rs | 641 ++++++++++-------- .../physical-expr/src/sort_properties.rs | 3 +- 2 files changed, 368 insertions(+), 276 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 64ec7d7f21b9..f25385bddf70 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -36,8 +36,6 @@ use indexmap::map::Entry; use indexmap::IndexMap; use itertools::Itertools; -const PRINT_ON: bool = false; - /// 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 @@ -1012,6 +1010,20 @@ impl EquivalenceProperties { projected_eqs.ordering_satisfy_requirement_helper2(reqs) } + /// Helper function to check whether the given sort requirements are satisfied by any of the + /// existing orderings. + fn ordering_satisfy_requirement_helper2(mut self, reqs: LexRequirementRef) -> bool { + // First, standardize the given requirement: + let normalized_reqs = self.normalize_sort_requirements(reqs); + for normalized_req in normalized_reqs { + if !self.ordering_satisfy_requirement_helper3(&normalized_req) { + return false; + } + self = self.add_constants(std::iter::once(normalized_req.expr)); + } + true + } + fn ordering_satisfy_requirement_helper3( &self, req: &PhysicalSortRequirement, @@ -1028,20 +1040,6 @@ impl EquivalenceProperties { } } - /// Helper function to check whether the given sort requirements are satisfied by any of the - /// existing orderings. - fn ordering_satisfy_requirement_helper2(mut self, reqs: LexRequirementRef) -> bool { - // First, standardize the given requirement: - let normalized_reqs = self.normalize_sort_requirements(reqs); - for normalized_req in normalized_reqs { - if !self.ordering_satisfy_requirement_helper3(&normalized_req) { - return false; - } - self = self.add_constants(std::iter::once(normalized_req.expr)); - } - true - } - /// Checks whether the `given`` sort requirements are equal or more specific /// than the `reference` sort requirements. pub fn requirements_compatible( @@ -1200,19 +1198,11 @@ impl EquivalenceProperties { self.eq_group.project_expr(projection_mapping, expr) } - fn get_leading_orderings( - &self, - ignore_exprs: &[Arc], - ) -> Vec { + fn get_leading_orderings(&self) -> Vec { let mut leading_orderings = vec![]; - let normalized_ignore_exprs = - self.eq_group.normalize_exprs(ignore_exprs.to_vec()); for ordering in self.normalized_oeq_class().iter() { for sort_expr in ordering { - if !expr_consists_of_ignored_exprs( - &normalized_ignore_exprs, - &sort_expr.expr, - ) { + if !self.is_expr_constant(&sort_expr.expr) { leading_orderings.push(sort_expr.clone()); break; } @@ -1221,127 +1211,92 @@ impl EquivalenceProperties { leading_orderings } - /// Projects `ordering` according to the given projection mapping. - /// If the resulting ordering is invalid after projection, returns `None`. - fn project_ordering_helper( - &self, - mapping: &ProjectionMapping, - 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, - // 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(&sort_expr.expr, mapping) - .map(|expr| PhysicalSortExpr { - expr, - options: sort_expr.options, - }) - }) - .collect::>(); - (!result.is_empty()).then_some(result) - } - - // TODO: Add a stop signal as return value. fn calc_ordered_exprs( &self, mapping: &ProjectionMapping, - ) -> Vec<(Vec, bool)> { - let mut new_orderings = vec![]; - let leading_orderings = self.get_leading_orderings(&self.constants); + ) -> (Vec, Vec) { + let mut new_orderings_continuing = vec![]; + let mut new_orderings_complete = vec![]; + let leading_orderings = self.get_leading_orderings(); for (source, target) in mapping.iter() { let expr_ordering = self.get_expr_ordering(source.clone()); - if PRINT_ON { - println!("expr_ordering: {expr_ordering:?}"); - println!("self.constants: {:?}", self.constants); - } - let sort_options = match expr_ordering.state { - SortProperties::Ordered(options) => Some(options), - // Assign an arbitrary ordering direction to singleton columns. - SortProperties::Singleton => Some(SortOptions { - descending: false, - nulls_first: false, - }), - SortProperties::Unordered => None, - }; - if let Some(options) = sort_options { - let sort_expr = PhysicalSortExpr { - expr: target.clone(), - options, + let sort_options = + if let SortProperties::Ordered(sort_options) = expr_ordering.state { + sort_options + } else { + // expression is not ordered check next expression in the projection mapping + continue; }; - let new_ordering = vec![sort_expr.clone()]; - let mut is_end = false; - if let Some(leaf_orderings) = - expr_ordering.leaf_orderings(&leading_orderings) - { - if let Some(projected_leaf_orderings) = - self.project_ordering_helper(mapping, &leaf_orderings) - { - // All of the leaves are projected. - if projected_leaf_orderings.len() == leaf_orderings.len() { - // TODO: If expression is known to be strictly monothonic. Then we do not need to append leaf ordering expressions - for suffix in projected_leaf_orderings - .iter() - .permutations(projected_leaf_orderings.len()) - { - let mut new_ordering = vec![sort_expr.clone()]; - let suffix = suffix.into_iter().cloned(); - new_ordering.extend(suffix); - let new_ordering = - collapse_lex_ordering(new_ordering, &self.constants); - if !new_orderings - .contains(&(new_ordering.clone(), is_end)) - { - new_orderings.push((new_ordering, is_end)); - } - } - } else { - is_end = true; - if PRINT_ON { - println!( - "send stop signal, expr_ordering: {:?}", - expr_ordering - ); + let sort_expr = PhysicalSortExpr { + expr: target.clone(), + options: sort_options, + }; + let new_ordering = vec![sort_expr.clone()]; + let mut is_complete = true; + if let Some(leaves_orderings) = + expr_ordering.leaves_orderings(&leading_orderings) + { + let projected_leaves_orderings = leaves_orderings + .iter() + .flat_map(|sort_expr| { + self.project_expr(&sort_expr.expr, mapping).map(|expr| { + PhysicalSortExpr { + expr, + options: sort_expr.options, } - // TODO: send stop signal. - } - } else { - is_end = true; - if PRINT_ON { - println!( - "send stop signal, expr_ordering: {:?}", - expr_ordering - ); + }) + }) + .collect::>(); + // All of the leaves are projected. + if projected_leaves_orderings.len() == leaves_orderings.len() { + is_complete = false; + // TODO: If expression is known to be strictly monothonic. Then we do not need to append leaf ordering expressions + for suffix in projected_leaves_orderings + .iter() + .permutations(projected_leaves_orderings.len()) + { + let mut new_ordering = vec![sort_expr.clone()]; + let suffix = suffix.into_iter().cloned(); + new_ordering.extend(suffix); + let new_ordering = + collapse_lex_ordering(new_ordering, &self.constants); + if !new_orderings_continuing.contains(&new_ordering) { + new_orderings_continuing.push(new_ordering); } - // TODO: send stop signal. } } - let new_ordering = collapse_lex_ordering(new_ordering, &self.constants); - if !new_orderings.contains(&(new_ordering.clone(), is_end)) { - new_orderings.push((new_ordering, is_end)); - } } - } - for leading_ordering in leading_orderings { - if let Some(projected_leading_ordering) = - self.project_ordering_helper(mapping, &[leading_ordering]) - { - if !new_orderings.contains(&(projected_leading_ordering.clone(), false)) { - new_orderings.push((projected_leading_ordering, false)); + if is_complete { + let new_ordering = collapse_lex_ordering(new_ordering, &self.constants); + if !new_orderings_complete.contains(&new_ordering) { + new_orderings_complete.push(new_ordering); } } } - if PRINT_ON { - for new_ordering in &new_orderings { - println!("new_ordering: {new_ordering:?}"); + + // Project existing leading orderings. If leading ordering is a+b + // and mapping is a as a_new, b as b_new; Projected leading ordering + // would be a_new+b_new. + let projected_leading_orderings = leading_orderings + .into_iter() + .flat_map(|sort_expr| { + self.eq_group + .project_expr(mapping, &sort_expr.expr) + .map(|expr| PhysicalSortExpr { + expr, + options: sort_expr.options, + }) + }) + .collect::>(); + for projected_leading_ordering in projected_leading_orderings { + let projected_leading_ordering = vec![projected_leading_ordering]; + if !new_orderings_continuing.contains(&projected_leading_ordering) { + new_orderings_continuing.push(projected_leading_ordering); } } - new_orderings.retain(|(ordering, _is_end)| !ordering.is_empty()); - new_orderings + new_orderings_complete.retain(|ordering| !ordering.is_empty()); + new_orderings_continuing.retain(|ordering| !ordering.is_empty()); + (new_orderings_complete, new_orderings_continuing) } /// Projects `ordering` according to the given projection mapping. @@ -1351,150 +1306,85 @@ impl EquivalenceProperties { mapping: &ProjectionMapping, 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, - // the result should be [a ASC], not [a ASC, c ASC], even if column c is - // valid after projection. - let mut orderings: Vec<(LexOrdering, bool)> = vec![(vec![], false)]; - let mut completed_orderings_holder: Vec = vec![]; + let mut completed_orderings: Vec = vec![]; + let mut continuing_orderings: Vec = vec![]; for sort_expr in ordering.iter() { - let new_orderings = self.calc_ordered_exprs(mapping); - let mut projected_constants = if let Some(projected_constants) = self - .constants - .iter() - .map(|expr| self.eq_group.project_expr(mapping, expr)) - .collect::>>() - { - projected_constants + let (new_orderings_complete, new_ordering_continuing) = + self.calc_ordered_exprs(mapping); + let projected_constants = self.projected_constants(mapping); + + // Update constants such that they treat left side of the lex ordering as constant during ordering discovery. + self = self.add_constants(std::iter::once(sort_expr.expr.clone())); + + let (relevant_orderings, non_relevant_orderings): ( + Vec, + Vec, + ) = if continuing_orderings.is_empty() { + (vec![vec![]], vec![vec![]]) } else { - // At least one of the constants cannot be projected. Hence continuing the iteration is not guaranteed to consider all referred columns. - // Stop iteration. - break; + // Partition available orderings so that they orderings that can accept new + // suffix ordering and that cannot accept are seperated. + continuing_orderings.into_iter().partition(|ordering| { + is_ordering_constant(&projected_constants, ordering) + }) }; - for (source, target) in mapping.iter() { - if expr_consists_of_ignored_exprs(&self.constants, source) - && !physical_exprs_contains(&projected_constants, source) - { - projected_constants.push(target.clone()); - } - } - if PRINT_ON { - for ordering in &orderings { - println!("ordering: {ordering:?}"); - } - } - - if new_orderings.is_empty() { - break; + completed_orderings.extend(non_relevant_orderings); + let new_completed_orderings = if new_orderings_complete.is_empty() { + relevant_orderings.clone() } else { - let relevant_indices = orderings - .iter() - .enumerate() - .flat_map(|(idx, (ordering, is_end))| { - if ordering_consists_of_ignored_exprs( - &projected_constants, - ordering, - ) && !*is_end - { - Some(idx) - } else { - None - } - }) - .collect::>(); - let n_rep = new_orderings.len(); - let orderings_len = orderings.len(); - for relevant_idx in &relevant_indices { - for _rep_idx in 0..n_rep { - orderings.push(orderings[*relevant_idx].clone()); - } - } + suffix_relevant_orderings( + relevant_orderings.clone(), + &new_orderings_complete, + ) + }; + completed_orderings.extend(new_completed_orderings); - let new_indices = (orderings_len - ..orderings_len + (n_rep) * relevant_indices.len()) - .collect::>(); + let new_continuing_orderings = + suffix_relevant_orderings(relevant_orderings, &new_ordering_continuing); + continuing_orderings = new_continuing_orderings; - for (iter_idx, ordering_idx) in new_indices.into_iter().enumerate() { - let mod_idx = iter_idx % new_orderings.len(); - let (new_ordering, is_end) = &new_orderings[mod_idx]; - orderings[ordering_idx].0.extend(new_ordering.to_vec()); - orderings[ordering_idx].1 = *is_end; - } - } - self = self.add_constants(std::iter::once(sort_expr.expr.clone())); - if PRINT_ON { - for ordering in &orderings { - println!("updated ordering: {ordering:?}"); - } - } - if PRINT_ON { - println!("orderings:{:?}", orderings); - } - let (completed_orderings, continuing_orderings): (Vec<_>, Vec<_>) = orderings - .into_iter() - .partition(|(_ordering, is_end)| *is_end); - let continuing_orderings = continuing_orderings - .into_iter() - .map(|(ordering, _is_end)| ordering) - .collect::>(); - let completed_orderings = completed_orderings - .into_iter() - .map(|(ordering, _is_end)| ordering) - .collect::>(); - if PRINT_ON { - println!("continuing_orderings:{:?}", continuing_orderings); - println!("completed_orderings:{:?}", completed_orderings); - } - completed_orderings_holder.extend(completed_orderings); - let continuing_orderings = continuing_orderings + continuing_orderings = continuing_orderings .into_iter() .map(|ordering| self.eq_group.normalize_sort_exprs(&ordering)) .collect(); let oeq_class = OrderingEquivalenceClass::new(continuing_orderings); - orderings = oeq_class - .orderings - .into_iter() - .map(|ordering| (ordering, false)) - .collect::<_>(); - if PRINT_ON { - for ordering in &orderings { - println!("updated ordering2: {ordering:?}"); - } + continuing_orderings = oeq_class.orderings; + if continuing_orderings.is_empty() { + // Cannot continue iteration safely. Subsequent orderings cannot be calculated properly. + break; } } - let mut orderings = orderings - .into_iter() - .map(|(ordering, _is_end)| ordering) - .collect::>(); - orderings.extend(completed_orderings_holder); + // Merge completed and continuing orderings. + let mut orderings = completed_orderings; + orderings.extend(continuing_orderings); let oeq_class = OrderingEquivalenceClass::new(orderings); orderings = oeq_class.orderings; + orderings.retain(|ordering| !ordering.is_empty()); + (!orderings.is_empty()).then_some(orderings) + } - let leading_ordering_exprs = orderings + fn projected_constants( + &self, + mapping: &ProjectionMapping, + ) -> Vec> { + // Project existing constants + // As an example assume that a+b is known to be constant. If projection were: + // `a as a_new`, `b as b_new`; we would project constant `a+b` as `a_new+b+new` + let mut projected_constants = self + .constants .iter() - .flat_map(|ordering| ordering.first().map(|sort_expr| sort_expr.expr.clone())) + .flat_map(|expr| self.eq_group.project_expr(mapping, expr)) .collect::>(); - - orderings = orderings - .into_iter() - .map(|ordering| { - let collapsed = collapse_lex_ordering(ordering, &leading_ordering_exprs); - self.eq_group.normalize_sort_exprs(&collapsed) - }) - .collect(); - if PRINT_ON { - println!("mapping:{:?}", mapping); - println!("leading_ordering_exprs: {:?}", leading_ordering_exprs); - println!("self.constants: {:?}", self.constants); - println!("self.eq_group: {:?}", self.eq_group); - for ordering in &orderings { - println!("updated ordering3: {ordering:?}"); + // Add projection expressions that are known to be constant. + for (source, target) in mapping.iter() { + if self.is_expr_constant(source) + && !physical_exprs_contains(&projected_constants, target) + { + projected_constants.push(target.clone()); } } - orderings.retain(|ordering| !ordering.is_empty()); - (!orderings.is_empty()).then_some(orderings) + projected_constants } /// Projects the equivalences within according to `projection_mapping` @@ -1504,11 +1394,7 @@ impl EquivalenceProperties { projection_mapping: &ProjectionMapping, output_schema: SchemaRef, ) -> Self { - let projected_constants = self - .constants - .iter() - .flat_map(|constant| projection_mapping.target_expr(constant)) - .collect::>(); + let projected_constants = self.projected_constants(projection_mapping); let projected_eq_group = self.eq_group.project(projection_mapping); let projected_orderings = self @@ -1594,7 +1480,7 @@ impl EquivalenceProperties { fn is_expr_constant(&self, expr: &Arc) -> bool { let normalized_constants = self.eq_group.normalize_exprs(self.constants.to_vec()); let normalized_expr = self.eq_group.normalize_expr(expr.clone()); - physical_exprs_contains(&normalized_constants, &normalized_expr) + is_expr_constant_util(&normalized_constants, &normalized_expr) } pub fn get_expr_ordering(&self, expr: Arc) -> ExprOrdering { @@ -1605,11 +1491,11 @@ impl EquivalenceProperties { } } -fn expr_consists_of_ignored_exprs( - ignored_exprs: &[Arc], +fn is_expr_constant_util( + constants: &[Arc], expr: &Arc, ) -> bool { - if physical_exprs_contains(ignored_exprs, expr) { + if physical_exprs_contains(constants, expr) { return true; } let children = expr.children(); @@ -1618,17 +1504,17 @@ fn expr_consists_of_ignored_exprs( } else { children .iter() - .all(|child| expr_consists_of_ignored_exprs(ignored_exprs, child)) + .all(|child| is_expr_constant_util(constants, child)) } } -fn ordering_consists_of_ignored_exprs( +fn is_ordering_constant( ignored_exprs: &[Arc], ordering: &[PhysicalSortExpr], ) -> bool { ordering .iter() - .all(|sort_expr| expr_consists_of_ignored_exprs(ignored_exprs, &sort_expr.expr)) + .all(|sort_expr| is_expr_constant_util(ignored_exprs, &sort_expr.expr)) } /// Calculate ordering equivalence properties for the given join operation. @@ -1767,6 +1653,21 @@ fn update_ordering( } } +fn suffix_relevant_orderings( + relevant_orderings: Vec, + new_orderings: &[LexOrdering], +) -> Vec { + let mut result = vec![]; + for relevant_ordering in &relevant_orderings { + for new_ordering in new_orderings { + let mut ordering = relevant_ordering.clone(); + ordering.extend(new_ordering.to_vec()); + result.push(ordering); + } + } + result +} + #[cfg(test)] mod tests { use std::ops::Not; @@ -2336,7 +2237,7 @@ mod tests { // Generate a data that satisfies properties given let table_data_with_properties = generate_table_for_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; - // print_batches(&[table_data_with_properties.clone()])?; + let exp_fn = create_physical_expr( &BuiltinScalarFunction::Floor, &[col("a", &test_schema)?], @@ -4022,9 +3923,9 @@ mod tests { // orderings vec![ // [a ASC, b ASC] - vec![(&col_a, option_asc), (&col_b, option_asc)], + vec![(col_a, option_asc), (col_b, option_asc)], // [a ASC, d ASC] - vec![(&col_a, option_asc), (&col_d, option_asc)], + vec![(col_a, option_asc), (col_d, option_asc)], ], // proj exprs vec![ @@ -4040,6 +3941,25 @@ mod tests { vec![("a_new", option_asc), ("b+d", option_asc)], ], ), + // ------- TEST CASE 13 ---------- + ( + // orderings + vec![ + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + ], + // proj exprs + vec![(col_c, "c_new".to_string()), (col_a, "a_new".to_string())], + // expected, This expected may be missing + vec![ + // [a_new ASC] + vec![("a_new", option_asc)], + ], + ), ]; for (orderings, proj_exprs, expected) in test_cases { @@ -4086,7 +4006,178 @@ mod tests { } #[test] - fn test_expr_consists_of_ignored_exprs() -> Result<()> { + fn project_orderings2() -> Result<()> { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + Field::new("ts", DataType::Timestamp(TimeUnit::Nanosecond, None), true), + ])); + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let col_c = &col("c", &schema)?; + let col_ts = &col("ts", &schema)?; + let a_plus_b = Arc::new(BinaryExpr::new( + col_a.clone(), + Operator::Plus, + col_b.clone(), + )) as Arc; + let interval = Arc::new(Literal::new(ScalarValue::IntervalDayTime(Some(2)))) + as Arc; + let date_bin_ts = &create_physical_expr( + &BuiltinScalarFunction::DateBin, + &[interval, col_ts.clone()], + &schema, + &ExecutionProps::default(), + )?; + + let round_c = &create_physical_expr( + &BuiltinScalarFunction::Round, + &[col_c.clone()], + &schema, + &ExecutionProps::default(), + )?; + + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + + let proj_exprs = vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (col_c, "c_new".to_string()), + (date_bin_ts, "date_bin_res".to_string()), + (round_c, "round_c_res".to_string()), + ]; + let proj_exprs = proj_exprs + .into_iter() + .map(|(expr, name)| (expr.clone(), name)) + .collect::>(); + let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &schema)?; + let output_schema = projection_mapping.output_schema(); + + let col_a_new = &col("a_new", &output_schema)?; + let col_b_new = &col("b_new", &output_schema)?; + let col_c_new = &col("c_new", &output_schema)?; + let col_date_bin_res = &col("date_bin_res", &output_schema)?; + let col_round_c_res = &col("round_c_res", &output_schema)?; + let a_new_plus_b_new = Arc::new(BinaryExpr::new( + col_a_new.clone(), + Operator::Plus, + col_b_new.clone(), + )) as Arc; + + let test_cases = vec![ + // ---------- TEST CASE 1 ------------ + ( + // orderings + vec![ + // [a ASC] + vec![(col_a, option_asc)], + ], + // expected + vec![ + // [b_new ASC] + vec![(col_a_new, option_asc)], + ], + ), + // ---------- TEST CASE 2 ------------ + ( + // orderings + vec![ + // [a+b ASC] + vec![(&a_plus_b, option_asc)], + ], + // expected + vec![ + // [b_new ASC] + vec![(&a_new_plus_b_new, option_asc)], + ], + ), + // ---------- TEST CASE 3 ------------ + ( + // orderings + vec![ + // [a ASC, ts ASC] + vec![(col_a, option_asc), (col_ts, option_asc)], + ], + // expected + vec![ + // [a_new ASC, date_bin_res ASC] + vec![(col_a_new, option_asc), (col_date_bin_res, option_asc)], + ], + ), + // ---------- TEST CASE 4 ------------ + ( + // orderings + vec![ + // [a ASC, ts ASC, b ASC] + vec![ + (col_a, option_asc), + (col_ts, option_asc), + (col_b, option_asc), + ], + ], + // expected + vec![ + // [a_new ASC, date_bin_res ASC] + // Please note that result is not [a_new ASC, date_bin_res ASC, b_new ASC] + // because, datebin_res may not be 1-1 function. Hence without introducing ts + // dependency we cannot guarantee any ordering after date_bin_res column. + vec![(col_a_new, option_asc), (col_date_bin_res, option_asc)], + ], + ), + // ---------- TEST CASE 5 ------------ + ( + // orderings + vec![ + // [a ASC, c ASC] + vec![(col_a, option_asc), (col_c, option_asc)], + ], + // expected + vec![ + // [a_new ASC, round_c_res ASC, c_new ASC] + vec![ + (col_a_new, option_asc), + (col_round_c_res, option_asc), + (col_c_new, option_asc), + ], + // [a_new ASC, c_new ASC] + vec![(col_a_new, option_asc), (col_c_new, option_asc)], + ], + ), + ]; + + for (orderings, expected) in test_cases { + let mut eq_properties = EquivalenceProperties::new(schema.clone()); + + let orderings = convert_to_orderings(&orderings); + eq_properties.add_new_orderings(orderings); + + let expected = convert_to_orderings(&expected); + + let projected_eq = + eq_properties.project(&projection_mapping, output_schema.clone()); + let orderings = projected_eq.oeq_class(); + + let err_msg = format!( + "actual: {:?}, expected: {:?}, projection_mapping: {:?}", + orderings.orderings, expected, projection_mapping + ); + + assert_eq!(orderings.len(), expected.len(), "{}", err_msg); + for expected_ordering in &expected { + assert!(orderings.contains(expected_ordering), "{}", err_msg) + } + } + + Ok(()) + } + + #[test] + fn test_expr_consists_of_constants() -> Result<()> { let schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::Int32, true), Field::new("b", DataType::Int32, true), @@ -4103,13 +4194,13 @@ mod tests { col_d.clone(), )) as Arc; - let ignored_exprs = vec![col_a.clone(), col_b.clone()]; + let constants = vec![col_a.clone(), col_b.clone()]; let expr = b_plus_d.clone(); - assert!(!expr_consists_of_ignored_exprs(&ignored_exprs, &expr)); + assert!(!is_expr_constant_util(&constants, &expr)); - let ignored_exprs = vec![col_a.clone(), col_b.clone(), col_d.clone()]; + let constants = vec![col_a.clone(), col_b.clone(), col_d.clone()]; let expr = b_plus_d.clone(); - assert!(expr_consists_of_ignored_exprs(&ignored_exprs, &expr)); + assert!(is_expr_constant_util(&constants, &expr)); Ok(()) } } diff --git a/datafusion/physical-expr/src/sort_properties.rs b/datafusion/physical-expr/src/sort_properties.rs index 1845226b0ae7..34440db55de8 100644 --- a/datafusion/physical-expr/src/sort_properties.rs +++ b/datafusion/physical-expr/src/sort_properties.rs @@ -167,7 +167,8 @@ impl ExprOrdering { } } - pub fn leaf_orderings( + // Returns either all of the leaves ordering or `None`. + pub fn leaves_orderings( &self, leading_orderings: &[PhysicalSortExpr], ) -> Option> { From 66d05d97c1d117f0c1f8d5492542179556386b79 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 16 Nov 2023 10:11:44 +0300 Subject: [PATCH 13/44] Minor changes --- .../replace_with_order_preserving_variants.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 6e08627b9f82..4ad9b14b8cd6 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 @@ -361,11 +361,11 @@ mod tests { " SortExec: expr=[a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true",]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true",]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) } From b78b9d7741e31265353bb59dc75d583a568481ad Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 16 Nov 2023 10:13:13 +0300 Subject: [PATCH 14/44] Minor changes --- .../replace_with_order_preserving_variants.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 4ad9b14b8cd6..7f8c9b852cb1 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 @@ -612,7 +612,7 @@ mod tests { " SortExec: expr=[c@1 ASC]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true",]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) } From 20b5f08d8bc823968716a07c6e074b2ef587d83c Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 16 Nov 2023 11:26:21 +0300 Subject: [PATCH 15/44] Minor changes --- datafusion/physical-expr/src/equivalence.rs | 22 ++++++++------------- 1 file changed, 8 insertions(+), 14 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 1d3e33aad539..4565657821ff 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1117,28 +1117,22 @@ impl EquivalenceProperties { /// This function applies an implicit projection to itself before calling `ordering_satisfy_requirement_helper`. /// This enables us to consider complex expressions during analysis. pub fn ordering_satisfy_requirement(&self, reqs: LexRequirementRef) -> bool { - let projected_eqs = self.clone(); - projected_eqs.ordering_satisfy_requirement_helper2(reqs) - } - - /// Helper function to check whether the given sort requirements are satisfied by any of the - /// existing orderings. - fn ordering_satisfy_requirement_helper2(mut self, reqs: LexRequirementRef) -> bool { + let mut eq_properties = self.clone(); // First, standardize the given requirement: - let normalized_reqs = self.normalize_sort_requirements(reqs); + let normalized_reqs = eq_properties.normalize_sort_requirements(reqs); for normalized_req in normalized_reqs { - if !self.ordering_satisfy_requirement_helper3(&normalized_req) { + if !eq_properties.ordering_satisfy_single_req(&normalized_req) { return false; } - self = self.add_constants(std::iter::once(normalized_req.expr)); + eq_properties = + eq_properties.add_constants(std::iter::once(normalized_req.expr)); } true } - fn ordering_satisfy_requirement_helper3( - &self, - req: &PhysicalSortRequirement, - ) -> bool { + /// Check whether PhysicalSortRequirement is satisfied by considering + /// leading orderings, equalities, and constant expressions. + fn ordering_satisfy_single_req(&self, req: &PhysicalSortRequirement) -> bool { let expr_ordering = self.get_expr_ordering(req.expr.clone()); let ExprOrdering { expr, state, .. } = expr_ordering; match state { From 3e1a2b99d44d19a41f965cecf681aa559a67700a Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 16 Nov 2023 13:36:31 +0300 Subject: [PATCH 16/44] All tests pass --- datafusion/physical-expr/src/equivalence.rs | 117 +++++++++++++++++--- 1 file changed, 101 insertions(+), 16 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 4565657821ff..e928e8d69c67 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -35,7 +35,7 @@ use datafusion_common::{JoinSide, JoinType, Result}; use indexmap::map::Entry; use indexmap::IndexMap; -use itertools::Itertools; +use itertools::{enumerate, Itertools}; /// 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 @@ -1533,13 +1533,31 @@ impl EquivalenceProperties { &self, exprs: &[Arc], ) -> (LexOrdering, Vec) { - let projection_mapping = self.implicit_projection_mapping(exprs); - let projected = self - .clone() - .project(&projection_mapping, projection_mapping.output_schema()); - projected.find_longest_permutation_helper( - &projection_mapping.target_exprs(exprs).unwrap_or_default(), - ) + let mut eq_properties = self.clone(); + let mut result_ordering = vec![]; + let mut indices = vec![]; + loop { + let mut is_fixed = true; + for (idx, expr) in exprs.iter().enumerate() { + // Do not add already added expressions + if !indices.contains(&idx){ + let ExprOrdering { expr, state, .. } = + eq_properties.get_expr_ordering(expr.clone()); + if let SortProperties::Ordered(options) = state { + eq_properties = + eq_properties.add_constants(std::iter::once(expr.clone())); + let sort_expr = PhysicalSortExpr { expr, options }; + result_ordering.push(sort_expr); + indices.push(idx); + is_fixed = false; + } + } + } + if is_fixed { + break; + } + } + (result_ordering, indices) } /// Helper function to calculate longest permutation. @@ -2383,13 +2401,6 @@ mod tests { // Check whether ordering_satisfy API result and // experimental result matches. - // assert!( - // !(eq_properties.ordering_satisfy(&requirement) == true - // && expected == false), - // "{}", - // err_msg - // ); - assert_eq!( eq_properties.ordering_satisfy(&requirement), (expected | false), @@ -3416,6 +3427,12 @@ mod tests { let col_d = &col("d", &test_schema)?; let col_e = &col("e", &test_schema)?; let col_h = &col("h", &test_schema)?; + // a + d + let a_plus_d = Arc::new(BinaryExpr::new( + col_a.clone(), + Operator::Plus, + col_d.clone(), + )) as Arc; let option_asc = SortOptions { descending: false, @@ -3445,8 +3462,8 @@ mod tests { ( vec![col_d, col_e, col_b], vec![ - (col_e, option_desc), (col_d, option_asc), + (col_e, option_desc), (col_b, option_asc), ], ), @@ -3454,6 +3471,13 @@ mod tests { (vec![col_b], vec![]), // TEST CASE 5 (vec![col_d], vec![(col_d, option_asc)]), + // TEST CASE 5 + (vec![&a_plus_d], vec![(&a_plus_d, option_asc)]), + // TEST CASE 6 + ( + vec![col_b, col_d], + vec![(col_d, option_asc), (col_b, option_asc)], + ), ]; for (exprs, expected) in test_cases { let exprs = exprs.into_iter().cloned().collect::>(); @@ -3465,6 +3489,67 @@ mod tests { Ok(()) } + #[test] + fn test_find_longest_permutation_random() -> Result<()> { + const N_RANDOM_SCHEMA: usize = 100; + 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 { + // Create a random schema with random properties + 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_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; + + let exp_fn = create_physical_expr( + &BuiltinScalarFunction::Floor, + &[col("a", &test_schema)?], + &test_schema, + &ExecutionProps::default(), + )?; + let a_plus_b = Arc::new(BinaryExpr::new( + col("a", &test_schema)?, + Operator::Plus, + col("b", &test_schema)?, + )) as Arc; + let 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)?, + exp_fn, + a_plus_b, + ]; + + for n_req in 0..=exprs.len() { + for exprs in exprs.iter().combinations(n_req) { + let exprs = exprs.into_iter().map(|item| item.clone()).collect::>(); + let (ordering, indices) = eq_properties.find_longest_permutation(&exprs); + + let err_msg = format!( + "Error in test case ordering:{:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}", + ordering, eq_properties.oeq_class, eq_properties.eq_group, eq_properties.constants + ); + // Since ordered section satisfies schema, we expect + // that result will be same after sort (e.g sort was unnecessary). + assert!(is_table_same_after_sort( + ordering.clone(), + table_data_with_properties.clone(), + )?); + } + } + } + + Ok(()) + } + #[test] fn test_update_ordering() -> Result<()> { let schema = Schema::new(vec![ From 9be89e23b071609de76ffdf4b53e88b1cc4a72b1 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 16 Nov 2023 13:42:30 +0300 Subject: [PATCH 17/44] Change implementation of find_longest_permutation --- datafusion/physical-expr/src/equivalence.rs | 104 +++----------------- 1 file changed, 13 insertions(+), 91 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index e928e8d69c67..1ac0d6e9356f 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -27,15 +27,12 @@ use crate::{ LexRequirement, LexRequirementRef, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; - use arrow::datatypes::SchemaRef; use arrow_schema::{Field, Schema, SortOptions}; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{JoinSide, JoinType, Result}; -use indexmap::map::Entry; -use indexmap::IndexMap; -use itertools::{enumerate, Itertools}; +use itertools::Itertools; /// 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 @@ -1245,43 +1242,6 @@ impl EquivalenceProperties { (!meet.is_empty()).then_some(meet) } - /// Creates a projection mapping to support complex expressions. - /// All of the existing fields + complex expressions - /// (expressions that needs to be evaluated using existing fields) such (a+b), (date_bin(ts), etc.) - /// are projected. - /// With this API, we can determine ordering properties of complex expressions without actually evaluating them. - fn implicit_projection_mapping( - &self, - exprs: &[Arc], - ) -> ProjectionMapping { - // Project existing fields as is - let mut proj_exprs = self - .schema - .fields - .iter() - .enumerate() - .map(|(idx, field)| { - let col = - Arc::new(Column::new(field.name(), idx)) as Arc; - (col, field.name().to_string()) - }) - .collect::>(); - // Project complex expression - let complex_proj_exprs = exprs - .iter() - .flat_map(|expr| { - if expr.as_any().is::() { - // Do not project column expressions - None - } else { - Some((expr.clone(), expr.to_string())) - } - }) - .collect::>(); - proj_exprs.extend(complex_proj_exprs); - ProjectionMapping::try_new(&proj_exprs, self.schema()).unwrap() - } - /// Projects argument `expr` according to `projection_mapping`, taking /// equivalences into account. /// @@ -1540,7 +1500,7 @@ impl EquivalenceProperties { let mut is_fixed = true; for (idx, expr) in exprs.iter().enumerate() { // Do not add already added expressions - if !indices.contains(&idx){ + if !indices.contains(&idx) { let ExprOrdering { expr, state, .. } = eq_properties.get_expr_ordering(expr.clone()); if let SortProperties::Ordered(options) = state { @@ -1560,45 +1520,6 @@ impl EquivalenceProperties { (result_ordering, indices) } - /// Helper function to calculate longest permutation. - fn find_longest_permutation_helper( - &self, - exprs: &[Arc], - ) -> (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(); - for ordering in self.normalized_oeq_class().iter() { - for sort_expr in ordering { - 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 { - // We only consider expressions that correspond to a prefix - // of one of the equivalent orderings we have. - break; - } - } - } - // Construct the lexicographical ordering according to the permutation: - ordered_exprs - .into_iter() - .map(|(idx, options)| { - ( - PhysicalSortExpr { - expr: exprs[idx].clone(), - options, - }, - idx, - ) - }) - .unzip() - } - fn is_expr_constant(&self, expr: &Arc) -> bool { let normalized_constants = self.eq_group.normalize_exprs(self.constants.to_vec()); let normalized_expr = self.eq_group.normalize_expr(expr.clone()); @@ -3494,10 +3415,6 @@ mod tests { const N_RANDOM_SCHEMA: usize = 100; 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 { // Create a random schema with random properties @@ -3530,8 +3447,9 @@ mod tests { for n_req in 0..=exprs.len() { for exprs in exprs.iter().combinations(n_req) { - let exprs = exprs.into_iter().map(|item| item.clone()).collect::>(); - let (ordering, indices) = eq_properties.find_longest_permutation(&exprs); + let exprs = exprs.into_iter().cloned().collect::>(); + let (ordering, _indices) = + eq_properties.find_longest_permutation(&exprs); let err_msg = format!( "Error in test case ordering:{:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}", @@ -3539,10 +3457,14 @@ mod tests { ); // Since ordered section satisfies schema, we expect // that result will be same after sort (e.g sort was unnecessary). - assert!(is_table_same_after_sort( - ordering.clone(), - table_data_with_properties.clone(), - )?); + assert!( + is_table_same_after_sort( + ordering.clone(), + table_data_with_properties.clone(), + )?, + "{}", + err_msg + ); } } } From 20e96d02c0127bced52c85ae7fbaf1ee2945fb2a Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 16 Nov 2023 13:47:51 +0300 Subject: [PATCH 18/44] Minor changes --- datafusion/physical-expr/src/equivalence.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 1ac0d6e9356f..8dcf497c567a 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -3399,6 +3399,11 @@ mod tests { vec![col_b, col_d], vec![(col_d, option_asc), (col_b, option_asc)], ), + // TEST CASE 6 + ( + vec![col_c, col_e], + vec![(col_c, option_asc), (col_e, option_desc)], + ), ]; for (exprs, expected) in test_cases { let exprs = exprs.into_iter().cloned().collect::>(); From 2400a9b50d19b185ca7319023fbe7fb4fe12d8f6 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 16 Nov 2023 13:54:48 +0300 Subject: [PATCH 19/44] Minor changes --- datafusion/physical-expr/src/equivalence.rs | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 8dcf497c567a..615709ae80a6 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -3453,13 +3453,27 @@ mod tests { for n_req in 0..=exprs.len() { for exprs in exprs.iter().combinations(n_req) { let exprs = exprs.into_iter().cloned().collect::>(); - let (ordering, _indices) = + let (ordering, indices) = eq_properties.find_longest_permutation(&exprs); + // Make sure that find_longest_permutation return values are consistent + let ordering2 = indices + .iter() + .zip(ordering.iter()) + .map(|(&idx, sort_expr)| PhysicalSortExpr { + expr: exprs[idx].clone(), + options: sort_expr.options, + }) + .collect::>(); + assert_eq!( + ordering, ordering2, + "indices and lexicographical ordering do not match" + ); let err_msg = format!( "Error in test case ordering:{:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}", ordering, eq_properties.oeq_class, eq_properties.eq_group, eq_properties.constants ); + assert_eq!(ordering.len(), indices.len(), "{}", err_msg); // Since ordered section satisfies schema, we expect // that result will be same after sort (e.g sort was unnecessary). assert!( From 7a1e03019457073c1bc6919de84b9d14dc802887 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Fri, 17 Nov 2023 08:59:00 +0300 Subject: [PATCH 20/44] Complex exprs requirement support (#215) * Discover ordering of complex expressions in group by and window partition by * Remove unnecessary tests * Update comments * Minor changes * Better projection support complex expression support * Fix failing test * Simplifications * Simplifications * Add is end flag * Simplifications * Simplifications * Simplifications * Minor changes * Minor changes * Minor changes * All tests pass * Change implementation of find_longest_permutation * Minor changes * Minor changes * Remove projection section * Remove projection implementation * Fix linter errors * Remove projection sections * Minor changes * Add docstring comments * Add comments * Minor changes * Minor changes * Add comments * simplifications --- .../enforce_distribution.rs | 8 +- datafusion/physical-expr/src/equivalence.rs | 1039 +++++++++++++---- .../physical-expr/src/sort_properties.rs | 1 - datafusion/physical-plan/src/projection.rs | 2 +- .../sqllogictest/test_files/groupby.slt | 45 + 5 files changed, 893 insertions(+), 202 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 4aedc3b0d1a9..a34958a6c96d 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -3787,7 +3787,7 @@ pub(crate) mod tests { fn repartition_transitively_past_sort_with_projection_and_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( @@ -3804,9 +3804,9 @@ pub(crate) mod tests { ); let expected = &[ - "SortPreservingMergeExec: [c@2 ASC]", + "SortPreservingMergeExec: [a@0 ASC]", // Expect repartition on the input to the sort (as it can benefit from additional parallelism) - "SortExec: expr=[c@2 ASC]", + "SortExec: expr=[a@0 ASC]", "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", "FilterExec: c@2 = 0", // repartition is lowest down @@ -3817,7 +3817,7 @@ pub(crate) mod tests { assert_optimized!(expected, plan.clone(), true); let expected_first_sort_enforcement = &[ - "SortExec: expr=[c@2 ASC]", + "SortExec: expr=[a@0 ASC]", "CoalescePartitionsExec", "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", "FilterExec: c@2 = 0", diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index f3bfe4961622..87031b12a949 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -15,26 +15,23 @@ // specific language governing permissions and limitations // under the License. -use std::collections::HashSet; use std::hash::Hash; use std::sync::Arc; -use crate::expressions::Column; +use crate::expressions::{Column, Literal}; +use crate::physical_expr::deduplicate_physical_exprs; use crate::sort_properties::{ExprOrdering, SortProperties}; use crate::{ physical_exprs_bag_equal, physical_exprs_contains, LexOrdering, LexOrderingRef, LexRequirement, LexRequirementRef, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; - use arrow::datatypes::SchemaRef; use arrow_schema::SortOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{JoinSide, JoinType, Result}; -use crate::physical_expr::deduplicate_physical_exprs; -use indexmap::map::Entry; -use indexmap::IndexMap; +use indexmap::IndexSet; /// 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 @@ -196,6 +193,32 @@ impl ProjectionMapping { ) -> impl Iterator, Arc)> + '_ { self.inner.iter() } + + /// This function returns target value for given expression + /// + /// # Arguments + /// + /// * `expr` - Source (e.g key) physical expression + /// + /// # Returns + /// + /// An `Option` containing a the target (e.g value) for the source expression. + /// `None` means that source is not found inside the mapping. + pub fn target_expr( + &self, + expr: &Arc, + ) -> Option> { + if let Some(idx) = self + .inner + .iter() + .position(|(source, _target)| source.eq(expr)) + { + let (_source, target) = &self.inner[idx]; + Some(target.clone()) + } else { + None + } + } } /// An `EquivalenceGroup` is a collection of `EquivalenceClass`es where each @@ -413,24 +436,32 @@ impl EquivalenceGroup { mapping: &ProjectionMapping, expr: &Arc, ) -> Option> { - let children = expr.children(); - if children.is_empty() { + // First, we try to project expressions with an exact match. If we are + // unable to do this, we consult equivalence classes. + if let Some(target) = mapping.target_expr(expr) { + // If we match the source, we can project directly: + return Some(target); + } else { + // If the given expression is not inside the mapping, try to project + // expressions considering the equivalence classes. for (source, target) in mapping.iter() { - // If we match the source, or an equivalent expression to source, + // If we match 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_equivalence_class(source) - .map_or(false, |group| group.contains(expr)) + // (a as a1, a + c) and the equivalence class (a, b), expression `b` projects to `a1`. + if self + .get_equivalence_class(source) + .map_or(false, |group| group.contains(expr)) { return Some(target.clone()); } } } // Project a non-leaf expression by projecting its children. - else if let Some(children) = children + let children = expr.children(); + if children.is_empty() { + // Leaf expression should be inside mapping. + return None; + } else if let Some(children) = children .into_iter() .map(|child| self.project_expr(mapping, &child)) .collect::>>() @@ -660,9 +691,27 @@ impl OrderingEquivalenceClass { /// 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) { + let leading_ordering_exprs = self + .orderings + .iter() + .flat_map(|ordering| ordering.first().map(|sort_expr| sort_expr.expr.clone())) + .collect::>(); + + // Remove leading orderings that are beyond index 0, to simplify ordering. + self.orderings.iter_mut().for_each(|ordering| { + let mut counter = 0; + ordering.retain(|sort_expr| { + // Either first entry or is not leading ordering + let should_retain = counter == 0 + || !physical_exprs_contains(&leading_ordering_exprs, &sort_expr.expr); + counter += 1; + should_retain + }); + }); + let mut idx = 0; while idx < self.orderings.len() { - let mut removal = false; + let mut removal = self.orderings[idx].is_empty(); for (ordering_idx, ordering) in self.orderings[0..idx].iter().enumerate() { if let Some(right_finer) = finer_side(ordering, &self.orderings[idx]) { if right_finer { @@ -985,36 +1034,56 @@ impl EquivalenceProperties { /// Checks whether the given sort requirements are satisfied by any of the /// existing orderings. pub fn ordering_satisfy_requirement(&self, reqs: LexRequirementRef) -> bool { + let mut eq_properties = self.clone(); // 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 normalized_reqs = eq_properties.normalize_sort_requirements(reqs); + for normalized_req in normalized_reqs { + // Check whether given ordering is satisfied + if !eq_properties.ordering_satisfy_single_req(&normalized_req) { + return false; + } + // - Treat satisfied ordering as constant in the next iterations. Since in lexicographical ordering + // next orderings are only considered as long as their left side have same values + // (e.g for them their left side is constant). + // + // Please note that, these expressions are not properly constant. This is just implementation + // and this interpretation doesn't effect outside users in anyway. + // + // As an example: + // If the requirement is `[a ASC, b + c ASC]`. + // and existing orderings are `[a ASC, b ASC], [c ASC]`. + // From the analysis above, we know that `[a ASC]` is satisfied. + // Then here, we add column `a` as constant to the state. + // This enables us to deduce that `b + c ASC` is satisfied, given `a` is constant. + eq_properties = + eq_properties.add_constants(std::iter::once(normalized_req.expr)); } - 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; - } - } + true + } + + /// Determines whether the ordering specified by a given `PhysicalSortRequirement` is satisfied + /// based on the internal orderings, equivalent classes, and constant expressions. + /// + /// # Arguments + /// + /// - `req`: A reference to a `PhysicalSortRequirement` for which the ordering satisfaction + /// needs to be determined. + /// + /// # Returns + /// + /// Returns `true` if the specified ordering is satisfied; otherwise, returns `false`. + fn ordering_satisfy_single_req(&self, req: &PhysicalSortRequirement) -> bool { + let expr_ordering = self.get_expr_ordering(req.expr.clone()); + let ExprOrdering { expr, state, .. } = expr_ordering; + match state { + SortProperties::Ordered(options) => { + let sort_expr = PhysicalSortExpr { expr, options }; + sort_expr.satisfy(req, self.schema()) } + // Singleton expressions satisfies any ordering. + SortProperties::Singleton => true, + SortProperties::Unordered => false, } - indices.len() == normalized_reqs.len() } /// Checks whether the `given`` sort requirements are equal or more specific @@ -1138,6 +1207,43 @@ impl EquivalenceProperties { self.eq_group.project_expr(projection_mapping, expr) } + /// Projects constants based on the provided `ProjectionMapping`. + /// + /// This function takes a `ProjectionMapping` and identifies and projects constants based on + /// the existing constants and the mapping. It ensures that constants are appropriately + /// propagated through the projection expressions. + /// + /// # Arguments + /// + /// - `mapping`: A reference to a `ProjectionMapping` representing the mapping of source + /// expressions to target expressions in the projection. + /// + /// # Returns + /// + /// Returns a vector of `Arc` containing the projected constants. + fn projected_constants( + &self, + mapping: &ProjectionMapping, + ) -> Vec> { + // Project existing constants + // As an example assume that a+b is known to be constant. If projection were: + // `a as a_new`, `b as b_new`; we would project constant `a+b` as `a_new+b_new` + let mut projected_constants = self + .constants + .iter() + .flat_map(|expr| self.eq_group.project_expr(mapping, expr)) + .collect::>(); + // Add projection expressions that are known to be constant. + for (source, target) in mapping.iter() { + if self.is_expr_constant(source) + && !physical_exprs_contains(&projected_constants, target) + { + projected_constants.push(target.clone()); + } + } + projected_constants + } + /// Projects the equivalences within according to `projection_mapping` /// and `output_schema`. pub fn project( @@ -1145,6 +1251,9 @@ impl EquivalenceProperties { projection_mapping: &ProjectionMapping, output_schema: SchemaRef, ) -> Self { + let projected_constants = self.projected_constants(projection_mapping); + let projected_eq_group = self.eq_group.project(projection_mapping); + let mut projected_orderings = self .oeq_class .iter() @@ -1152,7 +1261,8 @@ impl EquivalenceProperties { .collect::>(); for (source, target) in projection_mapping.iter() { let expr_ordering = ExprOrdering::new(source.clone()) - .transform_up(&|expr| update_ordering(expr, self)) + .transform_up(&|expr| Ok(update_ordering(expr, self))) + // It is guaranteed to always return Ok. .unwrap(); if let SortProperties::Ordered(options) = expr_ordering.state { // Push new ordering to the state. @@ -1163,9 +1273,9 @@ impl EquivalenceProperties { } } Self { - eq_group: self.eq_group.project(projection_mapping), + eq_group: projected_eq_group, oeq_class: OrderingEquivalenceClass::new(projected_orderings), - constants: vec![], + constants: projected_constants, schema: output_schema, } } @@ -1184,38 +1294,125 @@ impl EquivalenceProperties { &self, exprs: &[Arc], ) -> (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(); - for ordering in self.normalized_oeq_class().iter() { - for sort_expr in ordering { - 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); + let mut eq_properties = self.clone(); + let mut result = vec![]; + // Algorithm is as follows: + // - 1. Iterate over all expressions and insert the expressions that are known to be ordered + // into result_ordering. + // - 2. Treat inserted expressions as constants (add them as constant to the state) + // - 3. Go back to step 1. + // - Continue the above iteration as long as no new expression is inserted (Algorithm reached a fixed point). + + let mut search_indices = (0..exprs.len()).collect::>(); + // Algorithm above should reach a fixed point at-most `exprs.len()` number of iterations. + // We could have use loop{}, However, to guarantee we exit anyway (in-case of bugs). + // We use upper limit number of iterations (worst-case). + for _idx in 0..exprs.len() { + // Get ordered expressions with their indices. + let ordered_exprs = search_indices + .iter() + .flat_map(|&idx| { + let expr = exprs[idx].clone(); + let ExprOrdering { expr, state, .. } = + eq_properties.get_expr_ordering(expr); + if let SortProperties::Ordered(options) = state { + let sort_expr = PhysicalSortExpr { expr, options }; + Some((sort_expr, idx)) + } else { + None } - } else { - // We only consider expressions that correspond to a prefix - // of one of the equivalent orderings we have. - break; - } + }) + .collect::>(); + + // We reached a fixed point, exit. + if ordered_exprs.is_empty() { + break; } + + // - Remove indices that have an ordering from search_indices + // - Treat ordered expressions as constant in the next iterations. Since in lexicographical ordering + // next orderings are only considered as long as their left side have same values + // (e.g for them their left side is constant). + // + // Please note that, these expressions are not properly constant. This is just implementation + // and this interpretation doesn't effect outside users in anyway. + for (sort_expr, idx) in &ordered_exprs { + eq_properties = + eq_properties.add_constants(std::iter::once(sort_expr.expr.clone())); + search_indices.remove(idx); + } + + // Add new ordered section to the state. + result.extend(ordered_exprs); } - // Construct the lexicographical ordering according to the permutation: - ordered_exprs - .into_iter() - .map(|(idx, options)| { - ( - PhysicalSortExpr { - expr: exprs[idx].clone(), - options, - }, - idx, - ) - }) - .unzip() + result.into_iter().unzip() + } + + /// Checks whether a given expression is constant. + /// + /// This function determines whether the provided expression is constant based on the known constants. + /// + /// # Arguments + /// + /// - `expr`: A reference to a `Arc` representing the expression to be checked. + /// + /// # Returns + /// + /// Returns `true` if the expression is constant within the equivalence group; otherwise, returns `false`. + fn is_expr_constant(&self, expr: &Arc) -> bool { + // As an example, assume that we know columns [a, b] are constant. + // `a`, `b`, `a+b` will all return `true`, whereas `c` will return `false`. + let normalized_constants = self.eq_group.normalize_exprs(self.constants.to_vec()); + let normalized_expr = self.eq_group.normalize_expr(expr.clone()); + is_expr_constant_util(&normalized_constants, &normalized_expr) + } + + /// Retrieves the ordering information for a given physical expression. + /// + /// This function constructs an `ExprOrdering` object for the provided expression, which encapsulates + /// information about the expression's ordering, including its state and associated options. + /// + /// # Arguments + /// + /// - `expr`: An `Arc` representing the physical expression for which ordering information is sought. + /// + /// # Returns + /// + /// Returns an `ExprOrdering` object containing the ordering information for the given expression. + pub fn get_expr_ordering(&self, expr: Arc) -> ExprOrdering { + let expr_ordering = ExprOrdering::new(expr.clone()); + expr_ordering + .transform_up(&|expr| Ok(update_ordering(expr, self))) + // It is guaranteed to always return Ok. + .unwrap() + } +} + +/// Checks whether a given expression is constant. This function determines whether the +/// provided expression is constant based on the known constants. +/// +/// # Arguments +/// +/// - `constants`: A reference to a `[Arc]` containing expressions known to be a constant. +/// - `expr`: A reference to a `Arc` representing the expression to be checked. +/// +/// # Returns +/// +/// Returns `true` if the expression is constant within the equivalence group; otherwise, returns `false`. +fn is_expr_constant_util( + constants: &[Arc], + expr: &Arc, +) -> bool { + if physical_exprs_contains(constants, expr) { + return true; + } + let children = expr.children(); + if children.is_empty() { + false + } else { + children + .iter() + .all(|child| is_expr_constant_util(constants, child)) } } @@ -1330,26 +1527,28 @@ fn updated_right_ordering_equivalence_class( fn update_ordering( mut node: ExprOrdering, eq_properties: &EquivalenceProperties, -) -> Result> { +) -> Transformed { + // We have a Column, which is one of the two possible leaf node types: + let eq_group = &eq_properties.eq_group; + let normalized_expr = eq_group.normalize_expr(node.expr.clone()); + let oeq_class = &eq_properties.normalized_oeq_class(); + if eq_properties.is_expr_constant(&normalized_expr) { + node.state = SortProperties::Singleton; + return Transformed::Yes(node); + } else if let Some(options) = oeq_class.get_options(&normalized_expr) { + node.state = SortProperties::Ordered(options); + return Transformed::Yes(node); + } if !node.expr.children().is_empty() { // We have an intermediate (non-leaf) node, account for its children: node.state = node.expr.get_ordering(&node.children_states); - 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 = &eq_properties.eq_group; - let normalized_expr = eq_group.normalize_expr(node.expr.clone()); - let oeq_class = &eq_properties.oeq_class; - if let Some(options) = oeq_class.get_options(&normalized_expr) { - node.state = SortProperties::Ordered(options); - Ok(Transformed::Yes(node)) - } else { - Ok(Transformed::No(node)) - } - } else { + Transformed::Yes(node) + } else if node.expr.as_any().is::() { // We have a Literal, which is the other possible leaf node type: node.state = node.expr.get_ordering(&[]); - Ok(Transformed::Yes(node)) + Transformed::Yes(node) + } else { + Transformed::No(node) } } @@ -1359,14 +1558,16 @@ mod tests { use std::sync::Arc; use super::*; + use crate::execution_props::ExecutionProps; use crate::expressions::{col, lit, BinaryExpr, Column, Literal}; + use crate::functions::create_physical_expr; 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 arrow_array::{ArrayRef, Float64Array, RecordBatch, UInt32Array}; + use arrow_schema::{Fields, SortOptions, TimeUnit}; use datafusion_common::{Result, ScalarValue}; - use datafusion_expr::Operator; + use datafusion_expr::{BuiltinScalarFunction, Operator}; use itertools::{izip, Itertools}; use rand::rngs::StdRng; @@ -1432,12 +1633,12 @@ mod tests { // 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 a = Field::new("a", DataType::Float64, true); + let b = Field::new("b", DataType::Float64, true); + let c = Field::new("c", DataType::Float64, true); + let d = Field::new("d", DataType::Float64, true); + let e = Field::new("e", DataType::Float64, true); + let f = Field::new("f", DataType::Float64, true); let schema = Arc::new(Schema::new(vec![a, b, c, d, e, f])); Ok(schema) @@ -1602,19 +1803,20 @@ mod tests { Field::new("a4", DataType::Int64, true), ])); + // a as a1, a as a2, a as a3, a as a3 + let proj_exprs = vec![ + (col_a.clone(), "a1".to_string()), + (col_a.clone(), "a2".to_string()), + (col_a.clone(), "a3".to_string()), + (col_a.clone(), "a4".to_string()), + ]; + let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &input_schema)?; + // 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)?; let col_a4 = &col("a4", &out_schema)?; - let projection_mapping = ProjectionMapping { - inner: 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(&projection_mapping, out_schema); // At the output a1=a2=a3=a4 @@ -1631,6 +1833,10 @@ mod tests { #[test] fn test_ordering_satisfy() -> Result<()> { + let input_schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int64, true), + Field::new("b", DataType::Int64, true), + ])); let crude = vec![PhysicalSortExpr { expr: Arc::new(Column::new("a", 0)), options: SortOptions::default(), @@ -1646,13 +1852,12 @@ mod tests { }, ]; // finer ordering satisfies, crude ordering should return true - let empty_schema = &Arc::new(Schema::empty()); - let mut eq_properties_finer = EquivalenceProperties::new(empty_schema.clone()); + let mut eq_properties_finer = EquivalenceProperties::new(input_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 eq_properties_crude = EquivalenceProperties::new(empty_schema.clone()); + let mut eq_properties_crude = EquivalenceProperties::new(input_schema.clone()); eq_properties_crude.oeq_class.push(crude.clone()); assert!(!eq_properties_crude.ordering_satisfy(&finer)); Ok(()) @@ -1826,6 +2031,296 @@ mod tests { Ok(()) } + #[test] + fn test_ordering_satisfy_with_equivalence2() -> 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 floor_a = &create_physical_expr( + &BuiltinScalarFunction::Floor, + &[col("a", &test_schema)?], + &test_schema, + &ExecutionProps::default(), + )?; + let floor_f = &create_physical_expr( + &BuiltinScalarFunction::Floor, + &[col("f", &test_schema)?], + &test_schema, + &ExecutionProps::default(), + )?; + let exp_a = &create_physical_expr( + &BuiltinScalarFunction::Exp, + &[col("a", &test_schema)?], + &test_schema, + &ExecutionProps::default(), + )?; + let a_plus_b = Arc::new(BinaryExpr::new( + col_a.clone(), + Operator::Plus, + col_b.clone(), + )) as Arc; + let options = SortOptions { + descending: false, + nulls_first: false, + }; + + let test_cases = vec![ + // ------------ TEST CASE 1 ------------ + ( + // orderings + vec![ + // [a ASC, d ASC, b ASC] + vec![(col_a, options), (col_d, options), (col_b, options)], + // [c ASC] + vec![(col_c, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [a ASC, b ASC], requirement is not satisfied. + vec![(col_a, options), (col_b, options)], + // expected: requirement is not satisfied. + false, + ), + // ------------ TEST CASE 2 ------------ + ( + // orderings + vec![ + // [a ASC, c ASC, b ASC] + vec![(col_a, options), (col_c, options), (col_b, options)], + // [d ASC] + vec![(col_d, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [floor(a) ASC], + vec![(floor_a, options)], + // expected: requirement is satisfied. + true, + ), + // ------------ TEST CASE 2.1 ------------ + ( + // orderings + vec![ + // [a ASC, c ASC, b ASC] + vec![(col_a, options), (col_c, options), (col_b, options)], + // [d ASC] + vec![(col_d, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [floor(f) ASC], (Please note that a=f) + vec![(floor_f, options)], + // expected: requirement is satisfied. + true, + ), + // ------------ TEST CASE 3 ------------ + ( + // orderings + vec![ + // [a ASC, c ASC, b ASC] + vec![(col_a, options), (col_c, options), (col_b, options)], + // [d ASC] + vec![(col_d, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [a ASC, c ASC, a+b ASC], + vec![(col_a, options), (col_c, options), (&a_plus_b, options)], + // expected: requirement is satisfied. + true, + ), + // ------------ TEST CASE 4 ------------ + ( + // orderings + vec![ + // [a ASC, b ASC, c ASC, d ASC] + vec![ + (col_a, options), + (col_b, options), + (col_c, options), + (col_d, options), + ], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [floor(a) ASC, a+b ASC], + vec![(floor_a, options), (&a_plus_b, options)], + // expected: requirement is satisfied. + false, + ), + // ------------ TEST CASE 5 ------------ + ( + // orderings + vec![ + // [a ASC, b ASC, c ASC, d ASC] + vec![ + (col_a, options), + (col_b, options), + (col_c, options), + (col_d, options), + ], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [exp(a) ASC, a+b ASC], + vec![(exp_a, options), (&a_plus_b, options)], + // expected: requirement is not satisfied. + // TODO: If we know that exp function is 1-to-1 function. + // we could have deduced that above requirement is satisfied. + false, + ), + // ------------ TEST CASE 6 ------------ + ( + // orderings + vec![ + // [a ASC, d ASC, b ASC] + vec![(col_a, options), (col_d, options), (col_b, options)], + // [c ASC] + vec![(col_c, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [a ASC, d ASC, floor(a) ASC], + vec![(col_a, options), (col_d, options), (floor_a, options)], + // expected: requirement is satisfied. + true, + ), + // ------------ TEST CASE 7 ------------ + ( + // orderings + vec![ + // [a ASC, c ASC, b ASC] + vec![(col_a, options), (col_c, options), (col_b, options)], + // [d ASC] + vec![(col_d, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [a ASC, floor(a) ASC, a + b ASC], + vec![(col_a, options), (floor_a, options), (&a_plus_b, options)], + // expected: requirement is not satisfied. + false, + ), + // ------------ TEST CASE 8 ------------ + ( + // orderings + vec![ + // [a ASC, b ASC, c ASC] + vec![(col_a, options), (col_b, options), (col_c, options)], + // [d ASC] + vec![(col_d, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [a ASC, c ASC, floor(a) ASC, a + b ASC], + vec![ + (col_a, options), + (col_c, options), + (&floor_a, options), + (&a_plus_b, options), + ], + // expected: requirement is not satisfied. + false, + ), + // ------------ TEST CASE 9 ------------ + ( + // orderings + vec![ + // [a ASC, b ASC, c ASC, d ASC] + vec![ + (col_a, options), + (col_b, options), + (col_c, options), + (col_d, options), + ], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [a ASC, b ASC, c ASC, floor(a) ASC], + vec![ + (col_a, options), + (col_b, options), + (&col_c, options), + (&floor_a, options), + ], + // expected: requirement is satisfied. + true, + ), + // ------------ TEST CASE 10 ------------ + ( + // orderings + vec![ + // [d ASC, b ASC] + vec![(col_d, options), (col_b, options)], + // [c ASC, a ASC] + vec![(col_c, options), (col_a, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [c ASC, d ASC, a + b ASC], + vec![(col_c, options), (col_d, options), (&a_plus_b, options)], + // expected: requirement is satisfied. + true, + ), + ]; + + for (orderings, eq_group, constants, reqs, expected) in test_cases { + let err_msg = + format!("error in test orderings: {orderings:?}, eq_group: {eq_group:?}, constants: {constants:?}, reqs: {reqs:?}, expected: {expected:?}"); + let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); + let orderings = convert_to_orderings(&orderings); + eq_properties.add_new_orderings(orderings); + let eq_group = eq_group + .into_iter() + .map(|eq_class| { + let eq_classes = eq_class.into_iter().cloned().collect::>(); + EquivalenceClass::new(eq_classes) + }) + .collect::>(); + let eq_group = EquivalenceGroup::new(eq_group); + eq_properties.add_equivalence_group(eq_group); + + let constants = constants.into_iter().cloned(); + eq_properties = eq_properties.add_constants(constants); + + let reqs = convert_to_sort_exprs(&reqs); + assert_eq!( + eq_properties.ordering_satisfy(&reqs), + expected, + "{}", + err_msg + ); + } + + Ok(()) + } + #[test] fn test_ordering_satisfy_with_equivalence_random() -> Result<()> { const N_RANDOM_SCHEMA: usize = 5; @@ -1865,8 +2360,8 @@ mod tests { table_data_with_properties.clone(), )?; let err_msg = format!( - "Error in test case requirement:{:?}, expected: {:?}", - requirement, expected + "Error in test case requirement:{:?}, expected: {:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}", + requirement, expected, eq_properties.oeq_class, eq_properties.eq_group, eq_properties.constants ); // Check whether ordering_satisfy API result and // experimental result matches. @@ -1883,6 +2378,78 @@ mod tests { Ok(()) } + #[test] + fn test_ordering_satisfy_with_equivalence_complex_random() -> Result<()> { + const N_RANDOM_SCHEMA: usize = 100; + 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 { + // Create a random schema with random properties + 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_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; + + let exp_fn = create_physical_expr( + &BuiltinScalarFunction::Floor, + &[col("a", &test_schema)?], + &test_schema, + &ExecutionProps::default(), + )?; + let a_plus_b = Arc::new(BinaryExpr::new( + col("a", &test_schema)?, + Operator::Plus, + col("b", &test_schema)?, + )) as Arc; + let 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)?, + exp_fn, + a_plus_b, + ]; + + for n_req in 0..=exprs.len() { + for exprs in exprs.iter().combinations(n_req) { + let requirement = exprs + .into_iter() + .map(|expr| PhysicalSortExpr { + expr: expr.clone(), + 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:{:?}, expected: {:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}", + requirement, expected, eq_properties.oeq_class, eq_properties.eq_group, eq_properties.constants + ); + // Check whether ordering_satisfy API result and + // experimental result matches. + + assert_eq!( + eq_properties.ordering_satisfy(&requirement), + (expected | false), + "{}", + err_msg + ); + } + } + } + + Ok(()) + } + #[test] fn test_ordering_satisfy_different_lengths() -> Result<()> { let test_schema = create_test_schema()?; @@ -2111,6 +2678,30 @@ mod tests { ], ], ), + // ------- TEST CASE 5 --------- + // Empty ordering + ( + vec![vec![]], + // No ordering in the state (empty ordering is ignored). + vec![], + ), + // ------- TEST CASE 6 --------- + ( + // ORDERINGS GIVEN + vec![ + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + // [b ASC] + vec![(col_b, option_asc)], + ], + // EXPECTED orderings that is succinct. + vec![ + // [a ASC] + vec![(col_a, option_asc)], + // [b ASC] + vec![(col_b, option_asc)], + ], + ), ]; for (orderings, expected) in test_cases { let orderings = convert_to_orderings(&orderings); @@ -2216,13 +2807,16 @@ mod tests { let mut columns = batch.columns().to_vec(); // Create a new unique column - let n_row = batch.num_rows() as u64; - let unique_col = Arc::new(UInt64Array::from_iter_values(0..n_row)) as ArrayRef; + let n_row = batch.num_rows(); + let vals: Vec = (0..n_row).collect::>(); + let vals: Vec = vals.into_iter().map(|val| val as f64).collect(); + let unique_col = Arc::new(Float64Array::from_iter_values(vals)) 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 unique_field = + Arc::new(Field::new(unique_col_name, DataType::Float64, false)); let fields: Vec<_> = original_schema .fields() .iter() @@ -2241,17 +2835,17 @@ mod tests { }); // Convert the required ordering to a list of SortColumn - let sort_columns: Vec<_> = required_ordering + let sort_columns = 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(), + .map(|order_expr| { + let expr_result = order_expr.expr.evaluate(&new_batch)?; + let values = expr_result.into_array(new_batch.num_rows())?; + Ok(SortColumn { + values, options: Some(order_expr.options), }) }) - .collect(); + .collect::>>()?; // Check if the indices after sorting match the initial ordering let sorted_indices = lexsort_to_indices(&sort_columns, None)?; @@ -2292,18 +2886,18 @@ mod tests { // 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) + let values: Vec = (0..num_elems) + .map(|_| rng.gen_range(0..max_val) as f64 / 2.0) .collect(); - Arc::new(UInt64Array::from_iter_values(values)) + Arc::new(Float64Array::from_iter_values(values)) }; // Fill constant columns 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 = - Arc::new(UInt64Array::from_iter_values(vec![0; n_elem])) as ArrayRef; + let arr = Arc::new(Float64Array::from_iter_values(vec![0 as f64; n_elem])) + as ArrayRef; schema_vec[idx] = Some(arr); } @@ -2620,6 +3214,12 @@ mod tests { let col_d = &col("d", &test_schema)?; let col_e = &col("e", &test_schema)?; let col_h = &col("h", &test_schema)?; + // a + d + let a_plus_d = Arc::new(BinaryExpr::new( + col_a.clone(), + Operator::Plus, + col_d.clone(), + )) as Arc; let option_asc = SortOptions { descending: false, @@ -2650,14 +3250,26 @@ mod tests { vec![col_d, col_e, col_b], vec![ (col_d, option_asc), - (col_b, option_asc), (col_e, option_desc), + (col_b, option_asc), ], ), // TEST CASE 4 (vec![col_b], vec![]), // TEST CASE 5 (vec![col_d], vec![(col_d, option_asc)]), + // TEST CASE 5 + (vec![&a_plus_d], vec![(&a_plus_d, option_asc)]), + // TEST CASE 6 + ( + vec![col_b, col_d], + vec![(col_d, option_asc), (col_b, option_asc)], + ), + // TEST CASE 6 + ( + vec![col_c, col_e], + vec![(col_c, option_asc), (col_e, option_desc)], + ), ]; for (exprs, expected) in test_cases { let exprs = exprs.into_iter().cloned().collect::>(); @@ -2669,6 +3281,82 @@ mod tests { Ok(()) } + #[test] + fn test_find_longest_permutation_random() -> Result<()> { + const N_RANDOM_SCHEMA: usize = 100; + const N_ELEMENTS: usize = 125; + const N_DISTINCT: usize = 5; + + for seed in 0..N_RANDOM_SCHEMA { + // Create a random schema with random properties + 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_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; + + let exp_fn = create_physical_expr( + &BuiltinScalarFunction::Floor, + &[col("a", &test_schema)?], + &test_schema, + &ExecutionProps::default(), + )?; + let a_plus_b = Arc::new(BinaryExpr::new( + col("a", &test_schema)?, + Operator::Plus, + col("b", &test_schema)?, + )) as Arc; + let 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)?, + exp_fn, + a_plus_b, + ]; + + for n_req in 0..=exprs.len() { + for exprs in exprs.iter().combinations(n_req) { + let exprs = exprs.into_iter().cloned().collect::>(); + let (ordering, indices) = + eq_properties.find_longest_permutation(&exprs); + // Make sure that find_longest_permutation return values are consistent + let ordering2 = indices + .iter() + .zip(ordering.iter()) + .map(|(&idx, sort_expr)| PhysicalSortExpr { + expr: exprs[idx].clone(), + options: sort_expr.options, + }) + .collect::>(); + assert_eq!( + ordering, ordering2, + "indices and lexicographical ordering do not match" + ); + + let err_msg = format!( + "Error in test case ordering:{:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}", + ordering, eq_properties.oeq_class, eq_properties.eq_group, eq_properties.constants + ); + assert_eq!(ordering.len(), indices.len(), "{}", err_msg); + // Since ordered section satisfies schema, we expect + // that result will be same after sort (e.g sort was unnecessary). + assert!( + is_table_same_after_sort( + ordering.clone(), + table_data_with_properties.clone(), + )?, + "{}", + err_msg + ); + } + } + } + + Ok(()) + } + #[test] fn test_update_ordering() -> Result<()> { let schema = Schema::new(vec![ @@ -2726,11 +3414,14 @@ 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, &eq_properties))?; + let leading_orderings = eq_properties + .oeq_class() + .iter() + .flat_map(|ordering| ordering.first().cloned()) + .collect::>(); + let expr_ordering = eq_properties.get_expr_ordering(expr.clone()); let err_msg = format!( - "expr:{:?}, expected: {:?}, actual: {:?}", + "expr:{:?}, expected: {:?}, actual: {:?}, leading_orderings: {leading_orderings:?}", expr, expected, expr_ordering.state ); assert_eq!(expr_ordering.state, expected, "{}", err_msg); @@ -2921,74 +3612,30 @@ mod tests { } #[test] - fn project_empty_output_ordering() -> Result<()> { - 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 eq_properties = EquivalenceProperties::new(Arc::new(schema.clone())); - let ordering = vec![PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: SortOptions::default(), - }]; - eq_properties.add_new_orderings([ordering]); - let projection_mapping = ProjectionMapping { - inner: 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 orderings = eq_properties - .project(&projection_mapping, projection_schema) - .oeq_class() - .output_ordering() - .unwrap_or_default(); - - assert_eq!( - vec![PhysicalSortExpr { - expr: Arc::new(Column::new("b_new", 0)), - options: SortOptions::default(), - }], - orderings - ); - - let schema = Schema::new(vec![ + fn test_expr_consists_of_constants() -> Result<()> { + let schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::Int32, true), Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), - ]); - let eq_properties = EquivalenceProperties::new(Arc::new(schema)); - let projection_mapping = ProjectionMapping { - inner: 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), + Field::new("d", DataType::Int32, true), + Field::new("ts", DataType::Timestamp(TimeUnit::Nanosecond, None), true), ])); - let projected = eq_properties.project(&projection_mapping, projection_schema); - // After projection there is no ordering. - assert!(projected.oeq_class().output_ordering().is_none()); - + let col_a = col("a", &schema)?; + let col_b = col("b", &schema)?; + let col_d = col("d", &schema)?; + let b_plus_d = Arc::new(BinaryExpr::new( + col_b.clone(), + Operator::Plus, + col_d.clone(), + )) as Arc; + + let constants = vec![col_a.clone(), col_b.clone()]; + let expr = b_plus_d.clone(); + assert!(!is_expr_constant_util(&constants, &expr)); + + let constants = vec![col_a.clone(), col_b.clone(), col_d.clone()]; + let expr = b_plus_d.clone(); + assert!(is_expr_constant_util(&constants, &expr)); Ok(()) } } diff --git a/datafusion/physical-expr/src/sort_properties.rs b/datafusion/physical-expr/src/sort_properties.rs index a3b201f84e9d..2e72edc07c37 100644 --- a/datafusion/physical-expr/src/sort_properties.rs +++ b/datafusion/physical-expr/src/sort_properties.rs @@ -18,7 +18,6 @@ use std::{ops::Neg, sync::Arc}; use crate::PhysicalExpr; - use arrow_schema::SortOptions; use datafusion_common::tree_node::{TreeNode, VisitRecursion}; use datafusion_common::Result; diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index b8e2d0e425d4..06ff6bde9db3 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -38,10 +38,10 @@ use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::stats::Precision; use datafusion_common::Result; use datafusion_execution::TaskContext; +use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::expressions::{Literal, UnKnownColumn}; use datafusion_physical_expr::EquivalenceProperties; -use datafusion_physical_expr::equivalence::ProjectionMapping; use futures::stream::{Stream, StreamExt}; use log::trace; diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index 4438d69af306..192f87cc7007 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -3842,6 +3842,51 @@ ProjectionExec: expr=[SUM(alias1)@1 as SUM(DISTINCT t1.x), MAX(alias1)@2 as MAX( --------------------ProjectionExec: expr=[CAST(x@0 AS Float64) as CAST(t1.x AS Float64)t1.x, y@1 as y] ----------------------MemoryExec: partitions=1, partition_sizes=[1] +# create an unbounded table that contains ordered timestamp. +statement ok +CREATE UNBOUNDED EXTERNAL TABLE csv_with_timestamps ( + name VARCHAR, + ts TIMESTAMP +) +STORED AS CSV +WITH ORDER (ts DESC) +LOCATION '../core/tests/data/timestamps.csv' + +# below query should work in streaming mode. +query TT +EXPLAIN SELECT date_bin('15 minutes', ts) as time_chunks + FROM csv_with_timestamps + GROUP BY (date_bin('15 minutes', ts)) + ORDER BY time_chunks DESC + LIMIT 5 +---- +logical_plan +Limit: skip=0, fetch=5 +--Sort: time_chunks DESC NULLS FIRST, fetch=5 +----Projection: date_bin(Utf8("15 minutes"),csv_with_timestamps.ts) AS time_chunks +------Aggregate: groupBy=[[date_bin(IntervalMonthDayNano("900000000000"), csv_with_timestamps.ts) AS date_bin(Utf8("15 minutes"),csv_with_timestamps.ts)]], aggr=[[]] +--------TableScan: csv_with_timestamps projection=[ts] +physical_plan +GlobalLimitExec: skip=0, fetch=5 +--SortPreservingMergeExec: [time_chunks@0 DESC], fetch=5 +----ProjectionExec: expr=[date_bin(Utf8("15 minutes"),csv_with_timestamps.ts)@0 as time_chunks] +------AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("15 minutes"),csv_with_timestamps.ts)@0 as date_bin(Utf8("15 minutes"),csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted +--------CoalesceBatchesExec: target_batch_size=2 +----------SortPreservingRepartitionExec: partitioning=Hash([date_bin(Utf8("15 minutes"),csv_with_timestamps.ts)@0], 8), input_partitions=8, sort_exprs=date_bin(Utf8("15 minutes"),csv_with_timestamps.ts)@0 DESC +------------AggregateExec: mode=Partial, gby=[date_bin(900000000000, ts@0) as date_bin(Utf8("15 minutes"),csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted +--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +----------------StreamingTableExec: partition_sizes=1, projection=[ts], infinite_source=true, output_ordering=[ts@0 DESC] + +query P +SELECT date_bin('15 minutes', ts) as time_chunks + FROM csv_with_timestamps + GROUP BY (date_bin('15 minutes', ts)) + ORDER BY time_chunks DESC + LIMIT 5 +---- +2018-12-13T12:00:00 +2018-11-13T17:00:00 + statement ok drop table t1 From d39f5b20d316cf6f2abb64bf49fd3fa2f1ffcd6f Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 17 Nov 2023 10:17:54 +0300 Subject: [PATCH 21/44] Minor changes --- datafusion/physical-plan/src/projection.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 171197757efb..06ff6bde9db3 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -370,7 +370,7 @@ mod tests { use crate::expressions; use crate::test; - use arrow_schema::{DataType, Field, Schema}; + use arrow_schema::DataType; use datafusion_common::ScalarValue; #[tokio::test] From 1120488204d56b155a2e1d9cce727fb3c584898a Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 17 Nov 2023 10:21:30 +0300 Subject: [PATCH 22/44] Remove unused methods --- datafusion/physical-expr/src/equivalence.rs | 42 --------------------- 1 file changed, 42 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index bbba8057e121..5d23dfa06fed 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -195,48 +195,6 @@ impl ProjectionMapping { self.inner.iter() } - /// This function projects ordering requirement according to projection. - /// - /// # Arguments - /// - /// * `lex_req` - Lexicographical ordering requirement. - /// - /// # Returns - /// - /// An `Option` containing the Lexicographical projected ordering requirement. - pub fn project_lex_reqs(&self, lex_req: LexRequirementRef) -> Option { - lex_req - .iter() - .map(|sort_req| { - self.target_expr(&sort_req.expr) - .map(|expr| PhysicalSortRequirement { - expr, - options: sort_req.options, - }) - }) - .collect::>>() - } - - /// This function returns target value for each expression - /// - /// # Arguments - /// - /// * `exprs` - Source (e.g key) physical expressions - /// - /// # Returns - /// - /// An `Option` containing a the targets (e.g value) for the source each expression. - /// Returns `Some(Vec<_>)` if all of the expressions are source in the projection mapping. - pub fn target_exprs( - &self, - exprs: &[Arc], - ) -> Option>> { - exprs - .iter() - .map(|expr| self.target_expr(expr)) - .collect::>>() - } - /// This function returns target value for given expression /// /// # Arguments From d2a266795d192d9d94d10394f25f9101dfab330e Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 17 Nov 2023 10:26:27 +0300 Subject: [PATCH 23/44] Minor changes --- datafusion/physical-expr/src/equivalence.rs | 15 +-------------- 1 file changed, 1 insertion(+), 14 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 5d23dfa06fed..9d554179e615 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1009,8 +1009,6 @@ impl EquivalenceProperties { /// Checks whether the given sort requirements are satisfied by any of the /// existing orderings. - /// This function applies an implicit projection to itself before calling `ordering_satisfy_requirement_helper`. - /// This enables us to consider complex expressions during analysis. pub fn ordering_satisfy_requirement(&self, reqs: LexRequirementRef) -> bool { let mut eq_properties = self.clone(); // First, standardize the given requirement: @@ -1421,9 +1419,6 @@ impl EquivalenceProperties { /// definition of "partial permutation", see: /// /// - /// - /// This function applies an implicit projection to itself before calling `find_longest_permutation_helper`. - /// This enables us to consider complex expressions during analysis. pub fn find_longest_permutation( &self, exprs: &[Arc], @@ -2007,15 +2002,6 @@ mod tests { let input_properties = EquivalenceProperties::new(input_schema.clone()); let col_a = col("a", &input_schema)?; - // a as a1, a as a2, a as a3, a as a3 - let proj_exprs = vec![ - (col_a.clone(), "a1".to_string()), - (col_a.clone(), "a2".to_string()), - (col_a.clone(), "a3".to_string()), - (col_a.clone(), "a4".to_string()), - ]; - let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &input_schema)?; - let out_schema = output_schema(&projection_mapping, &input_schema)?; // a as a1, a as a2, a as a3, a as a3 let proj_exprs = vec![ @@ -2026,6 +2012,7 @@ mod tests { ]; let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &input_schema)?; + let out_schema = output_schema(&projection_mapping, &input_schema)?; // 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)?; From 7604c13c783261f3903257240cdf969d92170dc3 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 17 Nov 2023 13:50:58 +0300 Subject: [PATCH 24/44] Add random projection test --- datafusion/physical-expr/src/equivalence.rs | 214 ++++++++++++++------ 1 file changed, 148 insertions(+), 66 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 9d554179e615..48553d88bd4d 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -667,23 +667,33 @@ impl OrderingEquivalenceClass { /// 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) { - let leading_ordering_exprs = self - .orderings - .iter() - .flat_map(|ordering| ordering.first().map(|sort_expr| sort_expr.expr.clone())) - .collect::>(); - - // Remove leading orderings that are beyond index 0, to simplify ordering. - self.orderings.iter_mut().for_each(|ordering| { - let mut counter = 0; - ordering.retain(|sort_expr| { - // Either first entry or is not leading ordering - let should_retain = counter == 0 - || !physical_exprs_contains(&leading_ordering_exprs, &sort_expr.expr); - counter += 1; - should_retain - }); - }); + // TODO: we can remove leading orderings from the end + + // let leading_ordering_exprs = self + // .orderings + // .iter() + // .flat_map(|ordering| ordering.first().map(|sort_expr| sort_expr.expr.clone())) + // .collect::>(); + // + // // Remove leading orderings that are beyond index 0, to simplify ordering. + // self.orderings.iter_mut().for_each(|ordering| { + // while ordering.len() > 1{ + // let end_sort_expr = &ordering[ordering.len()-1]; + // if physical_exprs_contains(&leading_ordering_exprs, &end_sort_expr.expr){ + // ordering.pop(); + // } else { + // break; + // } + // } + // // let mut counter = 0; + // // ordering.retain(|sort_expr| { + // // // Either first entry or is not leading ordering + // // let should_retain = counter == 0 + // // || !physical_exprs_contains(&leading_ordering_exprs, &sort_expr.expr); + // // counter += 1; + // // should_retain + // // }); + // }); let mut idx = 0; while idx < self.orderings.len() { @@ -2596,7 +2606,7 @@ mod tests { let table_data_with_properties = generate_table_for_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; - let exp_fn = create_physical_expr( + let floor_a = create_physical_expr( &BuiltinScalarFunction::Floor, &[col("a", &test_schema)?], &test_schema, @@ -2614,7 +2624,7 @@ mod tests { col("d", &test_schema)?, col("e", &test_schema)?, col("f", &test_schema)?, - exp_fn, + floor_a, a_plus_b, ]; @@ -2897,8 +2907,8 @@ mod tests { ], // EXPECTED orderings that is succinct. vec![ - // [a ASC] - vec![(col_a, option_asc)], + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], // [b ASC] vec![(col_b, option_asc)], ], @@ -3495,7 +3505,7 @@ mod tests { let table_data_with_properties = generate_table_for_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; - let exp_fn = create_physical_expr( + let floor_a = create_physical_expr( &BuiltinScalarFunction::Floor, &[col("a", &test_schema)?], &test_schema, @@ -3513,7 +3523,7 @@ mod tests { col("d", &test_schema)?, col("e", &test_schema)?, col("f", &test_schema)?, - exp_fn, + floor_a, a_plus_b, ]; @@ -3900,8 +3910,8 @@ mod tests { ], // expected vec![ - // [date_bin_res ASC] - vec![("date_bin_res", option_asc)], + // [date_bin_res ASC, ts_new ASC] + vec![("date_bin_res", option_asc), ("ts_new", option_asc)], // [ts_new ASC] vec![("ts_new", option_asc)], ], @@ -3940,6 +3950,10 @@ mod tests { ("date_bin_res", option_asc), ("ts_new", option_asc), ], + // [a_new ASC, b_new ASC] + vec![("a_new", option_asc), ("b_new", option_asc)], + // [b_new ASC, a_new ASC] + vec![("b_new", option_asc), ("a_new", option_asc)], ], ), // ---------- TEST CASE 5 ------------ @@ -3982,42 +3996,6 @@ mod tests { ], ), // ------- TEST CASE 7 ---------- - ( - // orderings - vec![ - // [a ASC, ts ASC] - vec![(&col_a, option_asc), (&col_ts, option_asc)], - // [b ASC, ts ASC] - vec![(&col_b, option_asc), (&col_ts, option_asc)], - ], - // projection exprs - vec![ - (col_b, "b_new".to_string()), - (col_a, "a_new".to_string()), - (col_ts, "ts_new".to_string()), - (date_bin_func, "date_bin_res".to_string()), - ], - // expected - vec![ - // [a_new ASC, ts_new ASC] - vec![("a_new", option_asc), ("ts_new", option_asc)], - // [a_new ASC, date_bin_res ASC, ts_new ASC] - vec![ - ("a_new", option_asc), - ("date_bin_res", option_asc), - ("ts_new", option_asc), - ], - // [b_new ASC, ts_new ASC] - vec![("b_new", option_asc), ("ts_new", option_asc)], - // [b_new ASC, date_bin_res ASC, ts_new ASC] - vec![ - ("b_new", option_asc), - ("date_bin_res", option_asc), - ("ts_new", option_asc), - ], - ], - ), - // ------- TEST CASE 8 ---------- ( vec![ // [a ASC, b ASC, c ASC] @@ -4054,7 +4032,7 @@ mod tests { ], ], ), - // ------- TEST CASE 9 ---------- + // ------- TEST CASE 8 ---------- ( // orderings vec![ @@ -4074,7 +4052,7 @@ mod tests { vec![("b+d", option_asc)], ], ), - // ------- TEST CASE 10 ---------- + // ------- TEST CASE 9 ---------- ( // orderings vec![ @@ -4102,11 +4080,19 @@ mod tests { ("d_new", option_asc), ("b_new", option_asc), ], + // [a_new ASC, d_new ASC, c_new ASC] + vec![ + ("a_new", option_asc), + ("d_new", option_asc), + ("c_new", option_asc), + ], // [c_new ASC] vec![("c_new", option_asc)], + // [a_new ASC, c_new ASC] + vec![("a_new", option_asc), ("c_new", option_asc)], ], ), - // ------- TEST CASE 11 ---------- + // ------- TEST CASE 10 ---------- ( vec![ // [a ASC, b ASC, c ASC] @@ -4141,7 +4127,7 @@ mod tests { ], ], ), - // ------- TEST CASE 12 ---------- + // ------- TEST CASE 11 ---------- ( // orderings vec![ @@ -4164,7 +4150,7 @@ mod tests { vec![("a_new", option_asc), ("b+d", option_asc)], ], ), - // ------- TEST CASE 13 ---------- + // ------- TEST CASE 12 ---------- ( // orderings vec![ @@ -4374,6 +4360,25 @@ mod tests { vec![(col_a_new, option_asc), (col_c_new, option_asc)], ], ), + // ---------- TEST CASE 6 ------------ + ( + // orderings + vec![ + // [c ASC, b ASC] + vec![(col_c, option_asc), (col_b, option_asc)], + ], + // expected + vec![ + // [round_c_res ASC, c_new ASC] + vec![ + (col_round_c_res, option_asc), + (col_c_new, option_asc), + (col_b_new, option_asc), + ], + // [c_new ASC, b_new ASC] + vec![(col_c_new, option_asc), (col_b_new, option_asc)], + ], + ), ]; for (orderings, expected) in test_cases { @@ -4402,6 +4407,83 @@ mod tests { Ok(()) } + #[test] + fn project_orderings_random() -> Result<()> { + const N_RANDOM_SCHEMA: usize = 20; + const N_ELEMENTS: usize = 125; + const N_DISTINCT: usize = 5; + + for seed in 0..N_RANDOM_SCHEMA { + // Create a random schema with random properties + 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_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; + // Floor(a) + let floor_a = create_physical_expr( + &BuiltinScalarFunction::Floor, + &[col("a", &test_schema)?], + &test_schema, + &ExecutionProps::default(), + )?; + // a + b + let a_plus_b = Arc::new(BinaryExpr::new( + col("a", &test_schema)?, + Operator::Plus, + col("b", &test_schema)?, + )) as Arc; + let proj_exprs = vec![ + (col("a", &test_schema)?, "a_new"), + (col("b", &test_schema)?, "b_new"), + (col("c", &test_schema)?, "c_new"), + (col("d", &test_schema)?, "d_new"), + (col("e", &test_schema)?, "e_new"), + (col("f", &test_schema)?, "f_new"), + (floor_a, "floor(a)"), + (a_plus_b, "a+b"), + ]; + + for n_req in 0..=proj_exprs.len() { + for proj_exprs in proj_exprs.iter().combinations(n_req) { + let proj_exprs = proj_exprs.into_iter().map(|(expr, name)| (expr.clone(), name.to_string())).collect::>(); + let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &test_schema)?; + + let output_schema = output_schema(&projection_mapping, &test_schema)?; + + // Apply projection to the input record batch. + let projected_values = projection_mapping.iter().map(|(source, _target)| { + source.evaluate(&table_data_with_properties)?.into_array(N_ELEMENTS) + }).collect::>>()?; + let projected_batch = if projected_values.is_empty(){ + RecordBatch::new_empty(output_schema.clone()) + } else { + RecordBatch::try_new(output_schema.clone(), projected_values)? + }; + + let projected_eq = eq_properties.project(&projection_mapping, output_schema); + for ordering in projected_eq.oeq_class().iter(){ + let err_msg = format!( + "Error in test case ordering:{:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}, projection_mapping: {:?}", + ordering, eq_properties.oeq_class, eq_properties.eq_group, eq_properties.constants, projection_mapping + ); + // Since ordered section satisfies schema, we expect + // that result will be same after sort (e.g sort was unnecessary). + assert!( + is_table_same_after_sort( + ordering.clone(), + projected_batch.clone(), + )?, + "{}", + err_msg + ); + } + } + } + } + + Ok(()) + } + #[test] fn test_expr_consists_of_constants() -> Result<()> { let schema = Arc::new(Schema::new(vec![ From 792c8e97b001cbb87ef66f66aeaa00195bc04a54 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 17 Nov 2023 14:00:09 +0300 Subject: [PATCH 25/44] Minor changes --- datafusion/physical-expr/src/equivalence.rs | 93 +++++++++------------ 1 file changed, 40 insertions(+), 53 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 48553d88bd4d..2ba0bba1eaf0 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -667,33 +667,24 @@ impl OrderingEquivalenceClass { /// 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) { - // TODO: we can remove leading orderings from the end - - // let leading_ordering_exprs = self - // .orderings - // .iter() - // .flat_map(|ordering| ordering.first().map(|sort_expr| sort_expr.expr.clone())) - // .collect::>(); - // - // // Remove leading orderings that are beyond index 0, to simplify ordering. - // self.orderings.iter_mut().for_each(|ordering| { - // while ordering.len() > 1{ - // let end_sort_expr = &ordering[ordering.len()-1]; - // if physical_exprs_contains(&leading_ordering_exprs, &end_sort_expr.expr){ - // ordering.pop(); - // } else { - // break; - // } - // } - // // let mut counter = 0; - // // ordering.retain(|sort_expr| { - // // // Either first entry or is not leading ordering - // // let should_retain = counter == 0 - // // || !physical_exprs_contains(&leading_ordering_exprs, &sort_expr.expr); - // // counter += 1; - // // should_retain - // // }); - // }); + // Get leading orderings (e.g first sort expr in each lexicographical ordering) among orderings + let leading_ordering_exprs = self + .orderings + .iter() + .flat_map(|ordering| ordering.first().map(|sort_expr| sort_expr.expr.clone())) + .collect::>(); + + // Remove leading orderings that are at the end of the lexicographical ordering. + self.orderings.iter_mut().for_each(|ordering| { + while ordering.len() > 1 { + let end_sort_expr = &ordering[ordering.len() - 1]; + if physical_exprs_contains(&leading_ordering_exprs, &end_sort_expr.expr) { + ordering.pop(); + } else { + break; + } + } + }); let mut idx = 0; while idx < self.orderings.len() { @@ -1342,8 +1333,6 @@ impl EquivalenceProperties { .into_iter() .map(|ordering| self.eq_group.normalize_sort_exprs(&ordering)) .collect(); - let oeq_class = OrderingEquivalenceClass::new(continuing_orderings); - continuing_orderings = oeq_class.orderings; if continuing_orderings.is_empty() { // Cannot continue iteration safely. Subsequent orderings cannot be calculated properly. break; @@ -2907,8 +2896,8 @@ mod tests { ], // EXPECTED orderings that is succinct. vec![ - // [a ASC, b ASC] - vec![(col_a, option_asc), (col_b, option_asc)], + // [a ASC] + vec![(col_a, option_asc)], // [b ASC] vec![(col_b, option_asc)], ], @@ -3910,8 +3899,8 @@ mod tests { ], // expected vec![ - // [date_bin_res ASC, ts_new ASC] - vec![("date_bin_res", option_asc), ("ts_new", option_asc)], + // [date_bin_res ASC] + vec![("date_bin_res", option_asc)], // [ts_new ASC] vec![("ts_new", option_asc)], ], @@ -3950,10 +3939,6 @@ mod tests { ("date_bin_res", option_asc), ("ts_new", option_asc), ], - // [a_new ASC, b_new ASC] - vec![("a_new", option_asc), ("b_new", option_asc)], - // [b_new ASC, a_new ASC] - vec![("b_new", option_asc), ("a_new", option_asc)], ], ), // ---------- TEST CASE 5 ------------ @@ -4080,16 +4065,8 @@ mod tests { ("d_new", option_asc), ("b_new", option_asc), ], - // [a_new ASC, d_new ASC, c_new ASC] - vec![ - ("a_new", option_asc), - ("d_new", option_asc), - ("c_new", option_asc), - ], // [c_new ASC] vec![("c_new", option_asc)], - // [a_new ASC, c_new ASC] - vec![("a_new", option_asc), ("c_new", option_asc)], ], ), // ------- TEST CASE 10 ---------- @@ -4445,23 +4422,33 @@ mod tests { for n_req in 0..=proj_exprs.len() { for proj_exprs in proj_exprs.iter().combinations(n_req) { - let proj_exprs = proj_exprs.into_iter().map(|(expr, name)| (expr.clone(), name.to_string())).collect::>(); - let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &test_schema)?; + let proj_exprs = proj_exprs + .into_iter() + .map(|(expr, name)| (expr.clone(), name.to_string())) + .collect::>(); + let projection_mapping = + ProjectionMapping::try_new(&proj_exprs, &test_schema)?; let output_schema = output_schema(&projection_mapping, &test_schema)?; // Apply projection to the input record batch. - let projected_values = projection_mapping.iter().map(|(source, _target)| { - source.evaluate(&table_data_with_properties)?.into_array(N_ELEMENTS) - }).collect::>>()?; - let projected_batch = if projected_values.is_empty(){ + let projected_values = projection_mapping + .iter() + .map(|(source, _target)| { + source + .evaluate(&table_data_with_properties)? + .into_array(N_ELEMENTS) + }) + .collect::>>()?; + let projected_batch = if projected_values.is_empty() { RecordBatch::new_empty(output_schema.clone()) } else { RecordBatch::try_new(output_schema.clone(), projected_values)? }; - let projected_eq = eq_properties.project(&projection_mapping, output_schema); - for ordering in projected_eq.oeq_class().iter(){ + let projected_eq = + eq_properties.project(&projection_mapping, output_schema); + for ordering in projected_eq.oeq_class().iter() { let err_msg = format!( "Error in test case ordering:{:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}, projection_mapping: {:?}", ordering, eq_properties.oeq_class, eq_properties.eq_group, eq_properties.constants, projection_mapping From b35d343d0a7f59d7c1d7b67be324e2f32fb20820 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 17 Nov 2023 14:13:14 +0300 Subject: [PATCH 26/44] Minor changes --- datafusion/physical-expr/src/equivalence.rs | 29 ++++++++++++--------- 1 file changed, 16 insertions(+), 13 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 87031b12a949..a5ddb93a78f2 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -691,22 +691,25 @@ impl OrderingEquivalenceClass { /// 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) { + // Get leading orderings (e.g first sort expr in each lexicographical ordering) among orderings let leading_ordering_exprs = self .orderings .iter() .flat_map(|ordering| ordering.first().map(|sort_expr| sort_expr.expr.clone())) .collect::>(); - // Remove leading orderings that are beyond index 0, to simplify ordering. + // Remove leading orderings that are at the end of the lexicographical ordering. self.orderings.iter_mut().for_each(|ordering| { - let mut counter = 0; - ordering.retain(|sort_expr| { - // Either first entry or is not leading ordering - let should_retain = counter == 0 - || !physical_exprs_contains(&leading_ordering_exprs, &sort_expr.expr); - counter += 1; - should_retain - }); + while ordering.len() > 1 { + let last_sort_expr = &ordering[ordering.len() - 1]; + if physical_exprs_contains(&leading_ordering_exprs, &last_sort_expr.expr) + { + ordering.pop(); + } else { + // last ordering expr is not leading. Stop removing from the end. + break; + } + } }); let mut idx = 0; @@ -2395,7 +2398,7 @@ mod tests { let table_data_with_properties = generate_table_for_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; - let exp_fn = create_physical_expr( + let floor_a = create_physical_expr( &BuiltinScalarFunction::Floor, &[col("a", &test_schema)?], &test_schema, @@ -2413,7 +2416,7 @@ mod tests { col("d", &test_schema)?, col("e", &test_schema)?, col("f", &test_schema)?, - exp_fn, + floor_a, a_plus_b, ]; @@ -3294,7 +3297,7 @@ mod tests { let table_data_with_properties = generate_table_for_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; - let exp_fn = create_physical_expr( + let floor_a = create_physical_expr( &BuiltinScalarFunction::Floor, &[col("a", &test_schema)?], &test_schema, @@ -3312,7 +3315,7 @@ mod tests { col("d", &test_schema)?, col("e", &test_schema)?, col("f", &test_schema)?, - exp_fn, + floor_a, a_plus_b, ]; From 464c0a0926a8a1014a7420f10898d70b513c4a15 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 17 Nov 2023 14:39:32 +0300 Subject: [PATCH 27/44] Simplifications --- datafusion/physical-expr/src/equivalence.rs | 103 +++++------------- .../physical-expr/src/sort_properties.rs | 48 +------- 2 files changed, 30 insertions(+), 121 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 373181213100..b6180e74bcff 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -32,7 +32,6 @@ use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{JoinSide, JoinType, Result}; use indexmap::IndexSet; -use itertools::Itertools; /// 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 @@ -1206,6 +1205,10 @@ impl EquivalenceProperties { let mut new_orderings_continuing = vec![]; let mut new_orderings_complete = vec![]; let leading_orderings = self.get_leading_orderings(); + let leading_ordering_exprs = leading_orderings + .iter() + .map(|sort_expr| sort_expr.expr.clone()) + .collect::>(); for (source, target) in mapping.iter() { let expr_ordering = self.get_expr_ordering(source.clone()); let sort_options = @@ -1220,44 +1223,21 @@ impl EquivalenceProperties { options: sort_options, }; let new_ordering = vec![sort_expr.clone()]; - let mut is_complete = true; - if let Some(leaves_orderings) = - expr_ordering.leaves_orderings(&leading_orderings) - { - let projected_leaves_orderings = leaves_orderings - .iter() - .flat_map(|sort_expr| { - self.project_expr(&sort_expr.expr, mapping).map(|expr| { - PhysicalSortExpr { - expr, - options: sort_expr.options, - } - }) - }) - .collect::>(); - // All of the leaves are projected. - if projected_leaves_orderings.len() == leaves_orderings.len() { - is_complete = false; - // TODO: If expression is known to be strictly monothonic. Then we do not need to append leaf ordering expressions - for suffix in projected_leaves_orderings - .iter() - .permutations(projected_leaves_orderings.len()) - { - let mut new_ordering = vec![sort_expr.clone()]; - let suffix = suffix.into_iter().cloned(); - new_ordering.extend(suffix); - let new_ordering = collapse_lex_ordering(new_ordering); - if !new_orderings_continuing.contains(&new_ordering) { - new_orderings_continuing.push(new_ordering); - } - } - } - } - if is_complete { - let new_ordering = collapse_lex_ordering(new_ordering); - if !new_orderings_complete.contains(&new_ordering) { - new_orderings_complete.push(new_ordering); - } + + // expr is one of the leading ordering. This means that it is not a composite expression + // Hence its exactness doesn't depend on arguments. We can set is_complete true + // TODO: If we know that composite (complex) expression is 1-to-1 function. + // we can still continue iteration. If a is among leading orderings, exp(a) function + // can still continue. Even if it is not directly, among leading orderings. However, floor(a) cannot. + let can_continue = + physical_exprs_contains(&leading_ordering_exprs, &expr_ordering.expr); + let new_ordering = collapse_lex_ordering(new_ordering); + if can_continue && !new_orderings_continuing.contains(&new_ordering) { + // Can continue and isn't already added. + new_orderings_continuing.push(new_ordering); + } else if !can_continue && !new_orderings_complete.contains(&new_ordering) { + // Cannot continue and isn't already added. + new_orderings_complete.push(new_ordering); } } @@ -3927,20 +3907,12 @@ mod tests { vec![ // [a_new ASC, ts_new ASC] vec![("a_new", option_asc), ("ts_new", option_asc)], - // [a_new ASC, date_bin_res ASC, ts_new ASC] - vec![ - ("a_new", option_asc), - ("date_bin_res", option_asc), - ("ts_new", option_asc), - ], + // [a_new ASC, date_bin_res ASC] + vec![("a_new", option_asc), ("date_bin_res", option_asc)], // [b_new ASC, ts_new ASC] vec![("b_new", option_asc), ("ts_new", option_asc)], - // [b_new ASC, date_bin_res ASC, ts_new ASC] - vec![ - ("b_new", option_asc), - ("date_bin_res", option_asc), - ("ts_new", option_asc), - ], + // [b_new ASC, date_bin_res ASC] + vec![("b_new", option_asc), ("date_bin_res", option_asc)], ], ), // ---------- TEST CASE 5 ------------ @@ -4003,20 +3975,9 @@ mod tests { vec![("a_new", option_asc), ("b_new", option_asc)], // [a_new ASC, d_new ASC] vec![("a_new", option_asc), ("d_new", option_asc)], - // [a_new ASC, d_new ASC] + // [a_new ASC, b+d ASC] // TODO: In below case lexicographical argument ordering may be unnecessary for 1-to-1 functions. - vec![ - ("a_new", option_asc), - ("b+d", option_asc), - ("b_new", option_asc), - ("d_new", option_asc), - ], - vec![ - ("a_new", option_asc), - ("b+d", option_asc), - ("d_new", option_asc), - ("b_new", option_asc), - ], + vec![("a_new", option_asc), ("b+d", option_asc)], ], ), // ------- TEST CASE 8 ---------- @@ -4330,11 +4291,7 @@ mod tests { // expected vec![ // [a_new ASC, round_c_res ASC, c_new ASC] - vec![ - (col_a_new, option_asc), - (col_round_c_res, option_asc), - (col_c_new, option_asc), - ], + vec![(col_a_new, option_asc), (col_round_c_res, option_asc)], // [a_new ASC, c_new ASC] vec![(col_a_new, option_asc), (col_c_new, option_asc)], ], @@ -4348,12 +4305,8 @@ mod tests { ], // expected vec![ - // [round_c_res ASC, c_new ASC] - vec![ - (col_round_c_res, option_asc), - (col_c_new, option_asc), - (col_b_new, option_asc), - ], + // [round_c_res ASC] + vec![(col_round_c_res, option_asc)], // [c_new ASC, b_new ASC] vec![(col_c_new, option_asc), (col_b_new, option_asc)], ], diff --git a/datafusion/physical-expr/src/sort_properties.rs b/datafusion/physical-expr/src/sort_properties.rs index 4bfd57e8d4f9..2e72edc07c37 100644 --- a/datafusion/physical-expr/src/sort_properties.rs +++ b/datafusion/physical-expr/src/sort_properties.rs @@ -17,11 +17,11 @@ use std::{ops::Neg, sync::Arc}; -use crate::expressions::Literal; -use crate::{PhysicalExpr, PhysicalSortExpr}; +use crate::PhysicalExpr; use arrow_schema::SortOptions; use datafusion_common::tree_node::{TreeNode, VisitRecursion}; use datafusion_common::Result; + use itertools::Itertools; /// To propagate [`SortOptions`] across the [`PhysicalExpr`], it is insufficient @@ -166,17 +166,6 @@ impl ExprOrdering { } } - // Returns either all of the leaves ordering or `None`. - pub fn leaves_orderings( - &self, - leading_orderings: &[PhysicalSortExpr], - ) -> Option> { - let mut leaf_orderings = vec![]; - let completed = - leaf_orderings_helper(&self.expr, &mut leaf_orderings, leading_orderings); - completed.then_some(leaf_orderings) - } - /// Updates this [`ExprOrdering`]'s children states with the given states. pub fn with_new_children(mut self, children_states: Vec) -> Self { self.children_states = children_states; @@ -230,36 +219,3 @@ impl TreeNode for ExprOrdering { } } } - -fn leaf_orderings_helper( - expr: &Arc, - leaf_orderings: &mut Vec, - leading_orderings: &[PhysicalSortExpr], -) -> bool { - let children = expr.children(); - let mut state = SortProperties::Unordered; - for sort_expr in leading_orderings { - if expr.eq(&sort_expr.expr) { - state = SortProperties::Ordered(sort_expr.options); - break; - } - } - if let SortProperties::Ordered(options) = state { - let sort_expr = PhysicalSortExpr { - expr: expr.clone(), - options, - }; - leaf_orderings.push(sort_expr); - return true; - } - if expr.as_any().is::() { - return true; - } - if children.is_empty() { - false - } else { - children - .into_iter() - .all(|child| leaf_orderings_helper(&child, leaf_orderings, leading_orderings)) - } -} From cee9018c78315c1b943c17975b3d7549b55ad073 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 17 Nov 2023 15:18:44 +0300 Subject: [PATCH 28/44] Add comments --- datafusion/physical-expr/src/equivalence.rs | 56 ++++++++++++++------- 1 file changed, 39 insertions(+), 17 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index b6180e74bcff..433f5d2a760d 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1185,20 +1185,29 @@ impl EquivalenceProperties { self.eq_group.project_expr(projection_mapping, expr) } + /// Retrieves the leading orderings among all valid orderings. + /// + /// Leading ordering is the first non-constant ordering inside each + /// lexicographical ordering. + /// + /// # Returns + /// + /// A vector of `PhysicalSortExpr` representing the leading orderings. fn get_leading_orderings(&self) -> Vec { - let mut leading_orderings = vec![]; - for ordering in self.normalized_oeq_class().iter() { - for sort_expr in ordering { - if !self.is_expr_constant(&sort_expr.expr) { - leading_orderings.push(sort_expr.clone()); - break; - } - } - } - leading_orderings + self.normalized_oeq_class().iter().flat_map(|ordering |{ + // Get first non-constant sort_expr. + ordering.iter().find(| sort_expr| !self.is_expr_constant(&sort_expr.expr)).cloned() + }).collect() } - fn calc_ordered_exprs( + /// Retrieves the ordered expressions after projections + /// + /// # Returns + /// + /// A tuple of `Vec`, + /// - first entry contains orderings that cannot accept further suffix orderings. + /// - second entry contains orderings that can accept further suffix orderings. + fn get_ordered_exprs_after_projection( &self, mapping: &ProjectionMapping, ) -> (Vec, Vec) { @@ -1209,6 +1218,14 @@ impl EquivalenceProperties { .iter() .map(|sort_expr| sort_expr.expr.clone()) .collect::>(); + + // Find which projection expressions have an order after projection. + // Add ordered projection expressions to the either complete or continuing ordering expressions + // Complete ordering expressions means that, these orderings cannot accept suffix orderings after them. + // (Because they cannot guarantee how its arguments map during function, such as floor(a), round(a), etc.) + // This means that we cannot guarantee when floor(a) have a fixed value, a will also have a fixed value. + // Hence, ordering [x, a, b] shouldn't be projected as `[x, floor(a), b]` bu as `[x, floor(a)]` + // Continuing ordering expressions means that, these orderings can accept suffix orderings after them. for (source, target) in mapping.iter() { let expr_ordering = self.get_expr_ordering(source.clone()); let sort_options = @@ -1225,10 +1242,11 @@ impl EquivalenceProperties { let new_ordering = vec![sort_expr.clone()]; // expr is one of the leading ordering. This means that it is not a composite expression - // Hence its exactness doesn't depend on arguments. We can set is_complete true + // Hence its exactness doesn't depend on arguments. Then We can set can_continue true. // TODO: If we know that composite (complex) expression is 1-to-1 function. // we can still continue iteration. If a is among leading orderings, exp(a) function - // can still continue. Even if it is not directly, among leading orderings. However, floor(a) cannot. + // can still continue. Even if it is not directly, among leading orderings. However, + // after floor(a) cannot continue iteration. let can_continue = physical_exprs_contains(&leading_ordering_exprs, &expr_ordering.expr); let new_ordering = collapse_lex_ordering(new_ordering); @@ -1241,9 +1259,12 @@ impl EquivalenceProperties { } } - // Project existing leading orderings. If leading ordering is a+b - // and mapping is a as a_new, b as b_new; Projected leading ordering - // would be a_new+b_new. + // Project existing leading orderings. If leading ordering is `a+b` + // and mapping is `a as a_new`, `b as b_new`; Projected leading ordering + // would be `a_new+b_new`. + // Please note that: loop above where we iterate thorough all projections + // to understand ordering of the each projection expression cannot handles this case. + // Because neither `a_new` and `b_new` are themselves ordered. But their sum is ordered. let projected_leading_orderings = leading_orderings .into_iter() .flat_map(|sort_expr| { @@ -1255,6 +1276,7 @@ impl EquivalenceProperties { }) }) .collect::>(); + // Add projected leading orderings to the continuing orderings. for projected_leading_ordering in projected_leading_orderings { let projected_leading_ordering = vec![projected_leading_ordering]; if !new_orderings_continuing.contains(&projected_leading_ordering) { @@ -1277,7 +1299,7 @@ impl EquivalenceProperties { let mut continuing_orderings: Vec = vec![]; for sort_expr in ordering.iter() { let (new_orderings_complete, new_ordering_continuing) = - self.calc_ordered_exprs(mapping); + self.get_ordered_exprs_after_projection(mapping); let projected_constants = self.projected_constants(mapping); // Update constants such that they treat left side of the lex ordering as constant during ordering discovery. From 4659904184cc52784348f25a27d283b95039ff7b Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 17 Nov 2023 15:24:14 +0300 Subject: [PATCH 29/44] Simplifications --- 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 433f5d2a760d..96606086dc4e 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1194,10 +1194,16 @@ impl EquivalenceProperties { /// /// A vector of `PhysicalSortExpr` representing the leading orderings. fn get_leading_orderings(&self) -> Vec { - self.normalized_oeq_class().iter().flat_map(|ordering |{ - // Get first non-constant sort_expr. - ordering.iter().find(| sort_expr| !self.is_expr_constant(&sort_expr.expr)).cloned() - }).collect() + self.normalized_oeq_class() + .iter() + .flat_map(|ordering| { + // Get first non-constant sort_expr. + ordering + .iter() + .find(|sort_expr| !self.is_expr_constant(&sort_expr.expr)) + .cloned() + }) + .collect() } /// Retrieves the ordered expressions after projections @@ -1210,7 +1216,7 @@ impl EquivalenceProperties { fn get_ordered_exprs_after_projection( &self, mapping: &ProjectionMapping, - ) -> (Vec, Vec) { + ) -> (Vec, Vec) { let mut new_orderings_continuing = vec![]; let mut new_orderings_complete = vec![]; let leading_orderings = self.get_leading_orderings(); @@ -1239,7 +1245,7 @@ impl EquivalenceProperties { expr: target.clone(), options: sort_options, }; - let new_ordering = vec![sort_expr.clone()]; + // let new_ordering = vec![sort_expr.clone()]; // expr is one of the leading ordering. This means that it is not a composite expression // Hence its exactness doesn't depend on arguments. Then We can set can_continue true. @@ -1249,13 +1255,13 @@ impl EquivalenceProperties { // after floor(a) cannot continue iteration. let can_continue = physical_exprs_contains(&leading_ordering_exprs, &expr_ordering.expr); - let new_ordering = collapse_lex_ordering(new_ordering); - if can_continue && !new_orderings_continuing.contains(&new_ordering) { + // let new_ordering = collapse_lex_ordering(new_ordering); + if can_continue && !new_orderings_continuing.contains(&sort_expr) { // Can continue and isn't already added. - new_orderings_continuing.push(new_ordering); - } else if !can_continue && !new_orderings_complete.contains(&new_ordering) { + new_orderings_continuing.push(sort_expr); + } else if !can_continue && !new_orderings_complete.contains(&sort_expr) { // Cannot continue and isn't already added. - new_orderings_complete.push(new_ordering); + new_orderings_complete.push(sort_expr); } } @@ -1278,13 +1284,10 @@ impl EquivalenceProperties { .collect::>(); // Add projected leading orderings to the continuing orderings. for projected_leading_ordering in projected_leading_orderings { - let projected_leading_ordering = vec![projected_leading_ordering]; if !new_orderings_continuing.contains(&projected_leading_ordering) { new_orderings_continuing.push(projected_leading_ordering); } } - new_orderings_complete.retain(|ordering| !ordering.is_empty()); - new_orderings_continuing.retain(|ordering| !ordering.is_empty()); (new_orderings_complete, new_orderings_continuing) } @@ -1695,13 +1698,13 @@ fn update_ordering( fn suffix_relevant_orderings( relevant_orderings: Vec, - new_orderings: &[LexOrdering], + new_orderings: &[PhysicalSortExpr], ) -> Vec { let mut result = vec![]; for relevant_ordering in &relevant_orderings { for new_ordering in new_orderings { let mut ordering = relevant_ordering.clone(); - ordering.extend(new_ordering.to_vec()); + ordering.push(new_ordering.clone()); result.push(ordering); } } From 9f32fb79f0e05903a70cafa67e522c79b966967d Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 17 Nov 2023 16:41:33 +0300 Subject: [PATCH 30/44] Minor changes --- datafusion/physical-expr/src/equivalence.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 96606086dc4e..d3605b1e1d25 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1350,7 +1350,6 @@ impl EquivalenceProperties { orderings.extend(continuing_orderings); let oeq_class = OrderingEquivalenceClass::new(orderings); orderings = oeq_class.orderings; - orderings.retain(|ordering| !ordering.is_empty()); (!orderings.is_empty()).then_some(orderings) } From 0f5e7ecb71a5604e773ef97052ae86dd9b996389 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 17 Nov 2023 17:32:10 +0300 Subject: [PATCH 31/44] Simplifications --- datafusion/physical-expr/src/equivalence.rs | 85 +++++++++++---------- 1 file changed, 46 insertions(+), 39 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index d3605b1e1d25..9bb6ba3eb38f 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1217,8 +1217,6 @@ impl EquivalenceProperties { &self, mapping: &ProjectionMapping, ) -> (Vec, Vec) { - let mut new_orderings_continuing = vec![]; - let mut new_orderings_complete = vec![]; let leading_orderings = self.get_leading_orderings(); let leading_ordering_exprs = leading_orderings .iter() @@ -1226,44 +1224,41 @@ impl EquivalenceProperties { .collect::>(); // Find which projection expressions have an order after projection. - // Add ordered projection expressions to the either complete or continuing ordering expressions - // Complete ordering expressions means that, these orderings cannot accept suffix orderings after them. - // (Because they cannot guarantee how its arguments map during function, such as floor(a), round(a), etc.) - // This means that we cannot guarantee when floor(a) have a fixed value, a will also have a fixed value. - // Hence, ordering [x, a, b] shouldn't be projected as `[x, floor(a), b]` bu as `[x, floor(a)]` - // Continuing ordering expressions means that, these orderings can accept suffix orderings after them. - for (source, target) in mapping.iter() { - let expr_ordering = self.get_expr_ordering(source.clone()); - let sort_options = - if let SortProperties::Ordered(sort_options) = expr_ordering.state { - sort_options + let ordered_projections = mapping + .iter() + .flat_map(|(source, target)| { + let expr_ordering = self.get_expr_ordering(source.clone()); + if let SortProperties::Ordered(options) = expr_ordering.state { + Some(PhysicalSortExpr { + expr: target.clone(), + options, + }) } else { - // expression is not ordered check next expression in the projection mapping - continue; - }; - let sort_expr = PhysicalSortExpr { - expr: target.clone(), - options: sort_options, - }; - // let new_ordering = vec![sort_expr.clone()]; + // expression is not ordered, + None + } + }) + .collect::>(); - // expr is one of the leading ordering. This means that it is not a composite expression - // Hence its exactness doesn't depend on arguments. Then We can set can_continue true. + // Split ordered projection expressions to the either complete or continuing ordering expressions + // - Complete ordering expressions means that, these orderings cannot accept suffix orderings after them. + // (Because they cannot guarantee how its arguments map during function, such as floor(a), round(a), etc.) + // This means that we cannot guarantee when floor(a) have a fixed value, a will also have a fixed value. + // Hence, ordering [x, a, b] shouldn't be projected as `[x, floor(a), b]` bu as `[x, floor(a)]` + // - Continuing ordering expressions means that, these orderings can accept suffix orderings after them. + let (mut new_orderings_continuing, mut new_orderings_complete): ( + Vec, + Vec, + ) = ordered_projections.into_iter().partition(|sort_expr| { + // If expr is one of the leading ordering exprs. This means that it is not a composite expression + // Hence its exactness doesn't depend on arguments. These sort expression can accept suffix (e.g in continuing mode). + // Others cannot accept suffix (e.g in completed mode) // TODO: If we know that composite (complex) expression is 1-to-1 function. // we can still continue iteration. If a is among leading orderings, exp(a) function // can still continue. Even if it is not directly, among leading orderings. However, // after floor(a) cannot continue iteration. - let can_continue = - physical_exprs_contains(&leading_ordering_exprs, &expr_ordering.expr); - // let new_ordering = collapse_lex_ordering(new_ordering); - if can_continue && !new_orderings_continuing.contains(&sort_expr) { - // Can continue and isn't already added. - new_orderings_continuing.push(sort_expr); - } else if !can_continue && !new_orderings_complete.contains(&sort_expr) { - // Cannot continue and isn't already added. - new_orderings_complete.push(sort_expr); - } - } + physical_exprs_contains(&leading_ordering_exprs, &sort_expr.expr) + }); // Project existing leading orderings. If leading ordering is `a+b` // and mapping is `a as a_new`, `b as b_new`; Projected leading ordering @@ -1283,11 +1278,11 @@ impl EquivalenceProperties { }) .collect::>(); // Add projected leading orderings to the continuing orderings. - for projected_leading_ordering in projected_leading_orderings { - if !new_orderings_continuing.contains(&projected_leading_ordering) { - new_orderings_continuing.push(projected_leading_ordering); - } - } + new_orderings_continuing.extend(projected_leading_orderings); + + // De-duplicate orderings + new_orderings_complete = deduplicate_sort_exprs(new_orderings_complete); + new_orderings_continuing = deduplicate_sort_exprs(new_orderings_continuing); (new_orderings_complete, new_orderings_continuing) } @@ -1710,6 +1705,18 @@ fn suffix_relevant_orderings( result } +/// This function constructs a duplicate-free `Vec` by filtering out +/// duplicate entries that have same `PhysicalSortExpr` inside. +pub fn deduplicate_sort_exprs(input: Vec) -> Vec { + let mut output = Vec::::new(); + for item in input { + if !output.iter().any(|req| req.eq(&item)) { + output.push(item); + } + } + output +} + #[cfg(test)] mod tests { use std::ops::Not; From 6e2f7baf8cff660d4097ac7cbbc2d203c6d3d036 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 17 Nov 2023 17:46:00 +0300 Subject: [PATCH 32/44] Add new orderings --- datafusion/physical-expr/src/equivalence.rs | 46 +++++++++++++++++++++ 1 file changed, 46 insertions(+) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 9bb6ba3eb38f..0ac0d9d606c5 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -4140,6 +4140,52 @@ mod tests { vec![("a_new", option_asc)], ], ), + // ------- TEST CASE 13 ---------- + ( + // orderings + vec![ + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + // [a ASC, a + b ASC, c ASC] + vec![ + (col_a, option_asc), + (&a_plus_b, option_asc), + (col_c, option_asc), + ], + ], + // proj exprs + vec![ + (col_c, "c_new".to_string()), + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (&a_plus_b, "a+b".to_string()), + ], + // expected, This expected may be missing + vec![ + // [a_new ASC, b_new ASC, c_new ASC] + vec![ + ("a_new", option_asc), + ("b_new", option_asc), + ("c_new", option_asc), + ], + // [a_new ASC, a+b ASC, c_new ASC] + vec![ + ("a_new", option_asc), + ("a+b", option_asc), + ("c_new", option_asc), + ], + // [a_new ASC, a+b ASC, b_new ASC] + vec![ + ("a_new", option_asc), + ("a+b", option_asc), + ("b_new", option_asc), + ], + ], + ), ]; for (orderings, proj_exprs, expected) in test_cases { From 3a4bb761467af5d50512d68a4c3e66b5fbf477e2 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 17 Nov 2023 18:04:44 +0300 Subject: [PATCH 33/44] Minor changes --- datafusion/physical-expr/src/equivalence.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 0ac0d9d606c5..5f1f0654a47b 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1327,9 +1327,8 @@ impl EquivalenceProperties { }; completed_orderings.extend(new_completed_orderings); - let new_continuing_orderings = + continuing_orderings = suffix_relevant_orderings(relevant_orderings, &new_ordering_continuing); - continuing_orderings = new_continuing_orderings; continuing_orderings = continuing_orderings .into_iter() From 39ff1525ce87fc8821fe62cc00faa832602abb84 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 17 Nov 2023 18:06:31 +0300 Subject: [PATCH 34/44] Add comments --- datafusion/physical-expr/src/equivalence.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 5f1f0654a47b..d325e16bc255 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1689,6 +1689,7 @@ fn update_ordering( } } +/// Suffix each new_ordering to the end of the each relevant ordering. fn suffix_relevant_orderings( relevant_orderings: Vec, new_orderings: &[PhysicalSortExpr], From 2b8dffc31e53388439f94066b9f5cf4999f1ef67 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 20 Nov 2023 10:41:07 +0300 Subject: [PATCH 35/44] Add new tests --- datafusion/physical-expr/src/equivalence.rs | 148 +++++++++++++++++--- 1 file changed, 125 insertions(+), 23 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index d325e16bc255..87c5e0069cde 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1946,6 +1946,33 @@ mod tests { .collect() } + // Apply projection to the input_data, return projected equivalence properties and record batch + fn apply_projection( + proj_exprs: Vec<(Arc, String)>, + input_data: &RecordBatch, + input_eq_properties: &EquivalenceProperties, + ) -> Result<(RecordBatch, EquivalenceProperties)> { + let input_schema = input_data.schema(); + let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &input_schema)?; + + let output_schema = output_schema(&projection_mapping, &input_schema)?; + let num_rows = input_data.num_rows(); + // Apply projection to the input record batch. + let projected_values = projection_mapping + .iter() + .map(|(source, _target)| source.evaluate(input_data)?.into_array(num_rows)) + .collect::>>()?; + let projected_batch = if projected_values.is_empty() { + RecordBatch::new_empty(output_schema.clone()) + } else { + RecordBatch::try_new(output_schema.clone(), projected_values)? + }; + + let projected_eq = + input_eq_properties.project(&projection_mapping, output_schema); + Ok((projected_batch, projected_eq)) + } + #[test] fn add_equal_conditions_test() -> Result<()> { let schema = Arc::new(Schema::new(vec![ @@ -4416,6 +4443,96 @@ mod tests { Ok(()) } + #[test] + fn project_orderings3() -> Result<()> { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + Field::new("ts", DataType::Timestamp(TimeUnit::Nanosecond, None), true), + ])); + 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 a_plus_b = Arc::new(BinaryExpr::new( + col_a.clone(), + Operator::Plus, + col_b.clone(), + )) as Arc; + + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + + let proj_exprs = vec![ + (col_c, "c_new".to_string()), + (col_d, "d_new".to_string()), + (&a_plus_b, "a+b".to_string()), + ]; + let proj_exprs = proj_exprs + .into_iter() + .map(|(expr, name)| (expr.clone(), name)) + .collect::>(); + let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &schema)?; + let output_schema = output_schema(&projection_mapping, &schema)?; + + let _col_a_plus_b_new = &col("a+b", &output_schema)?; + let col_c_new = &col("c_new", &output_schema)?; + let col_d_new = &col("d_new", &output_schema)?; + + let test_cases = vec![ + // ---------- TEST CASE 1 ------------ + ( + // orderings + vec![ + // [d ASC, b ASC] + vec![(col_d, option_asc), (col_b, option_asc)], + // [c ASC, a ASC] + vec![(col_c, option_asc), (col_a, option_asc)], + ], + // expected + vec![ + // TODO: Ordering below is also valid. We should be able to understand it is valid. + // For now, we cannot understand it is valid. + // // [d_new ASC, c_new ASC, a+b ASC] + // vec![(col_d_new, option_asc), (col_c_new, option_asc), (_col_a_plus_b_new, option_asc)], + // [d_new ASC] + vec![(col_d_new, option_asc)], + // [c_new ASC] + vec![(col_c_new, option_asc)], + ], + ), + ]; + + for (orderings, expected) in test_cases { + let mut eq_properties = EquivalenceProperties::new(schema.clone()); + + let orderings = convert_to_orderings(&orderings); + eq_properties.add_new_orderings(orderings); + + let expected = convert_to_orderings(&expected); + + let projected_eq = + eq_properties.project(&projection_mapping, output_schema.clone()); + let orderings = projected_eq.oeq_class(); + + let err_msg = format!( + "actual: {:?}, expected: {:?}, projection_mapping: {:?}", + orderings.orderings, expected, projection_mapping + ); + + assert_eq!(orderings.len(), expected.len(), "{}", err_msg); + for expected_ordering in &expected { + assert!(orderings.contains(expected_ordering), "{}", err_msg) + } + } + + Ok(()) + } + #[test] fn project_orderings_random() -> Result<()> { const N_RANDOM_SCHEMA: usize = 20; @@ -4458,32 +4575,17 @@ mod tests { .into_iter() .map(|(expr, name)| (expr.clone(), name.to_string())) .collect::>(); - let projection_mapping = - ProjectionMapping::try_new(&proj_exprs, &test_schema)?; - - let output_schema = output_schema(&projection_mapping, &test_schema)?; - - // Apply projection to the input record batch. - let projected_values = projection_mapping - .iter() - .map(|(source, _target)| { - source - .evaluate(&table_data_with_properties)? - .into_array(N_ELEMENTS) - }) - .collect::>>()?; - let projected_batch = if projected_values.is_empty() { - RecordBatch::new_empty(output_schema.clone()) - } else { - RecordBatch::try_new(output_schema.clone(), projected_values)? - }; + let (projected_batch, projected_eq) = apply_projection( + proj_exprs.clone(), + &table_data_with_properties, + &eq_properties, + )?; - let projected_eq = - eq_properties.project(&projection_mapping, output_schema); + // Make sure each ordering after projection is valid. for ordering in projected_eq.oeq_class().iter() { let err_msg = format!( - "Error in test case ordering:{:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}, projection_mapping: {:?}", - ordering, eq_properties.oeq_class, eq_properties.eq_group, eq_properties.constants, projection_mapping + "Error in test case ordering:{:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}, proj_exprs: {:?}", + ordering, eq_properties.oeq_class, eq_properties.eq_group, eq_properties.constants, proj_exprs ); // Since ordered section satisfies schema, we expect // that result will be same after sort (e.g sort was unnecessary). From 9afd4b5e7540d112f65c144728d594ed85265f19 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 20 Nov 2023 14:12:17 +0300 Subject: [PATCH 36/44] Change project ordering implementation --- datafusion/physical-expr/src/equivalence.rs | 723 +++++++++++++------- 1 file changed, 471 insertions(+), 252 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 87c5e0069cde..48bd6b327028 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -32,6 +32,9 @@ use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{JoinSide, JoinType, Result}; use indexmap::IndexSet; +use itertools::{izip, Itertools}; + +const PRINT_ON: bool = false; /// 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 @@ -662,47 +665,35 @@ impl OrderingEquivalenceClass { self.remove_redundant_entries(); } - /// 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. + /// 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) { - // Get leading orderings (e.g first sort expr in each lexicographical ordering) among orderings - let leading_ordering_exprs = self - .orderings - .iter() - .flat_map(|ordering| ordering.first().map(|sort_expr| sort_expr.expr.clone())) - .collect::>(); - - // Remove leading orderings that are at the end of the lexicographical ordering. - self.orderings.iter_mut().for_each(|ordering| { - while ordering.len() > 1 { - let last_sort_expr = &ordering[ordering.len() - 1]; - if physical_exprs_contains(&leading_ordering_exprs, &last_sort_expr.expr) - { - ordering.pop(); - } else { - // last ordering expr is not leading. Stop removing from the end. - break; - } - } - }); - - let mut idx = 0; - while idx < self.orderings.len() { - let mut removal = self.orderings[idx].is_empty(); - for (ordering_idx, ordering) in self.orderings[0..idx].iter().enumerate() { - if let Some(right_finer) = finer_side(ordering, &self.orderings[idx]) { - if right_finer { - self.orderings.swap(ordering_idx, idx); + let mut work = true; + while work { + work = false; + let mut idx = 0; + while idx < self.orderings.len() { + let mut ordering_idx = idx + 1; + let mut removal = self.orderings[idx].is_empty(); + while ordering_idx < self.orderings.len() { + work |= resolve_overlap(&mut self.orderings, idx, ordering_idx); + if self.orderings[idx].is_empty() { + removal = true; + break; + } + work |= resolve_overlap(&mut self.orderings, ordering_idx, idx); + if self.orderings[ordering_idx].is_empty() { + self.orderings.swap_remove(ordering_idx); + } else { + ordering_idx += 1; } - removal = true; - break; } - } - if removal { - self.orderings.swap_remove(idx); - } else { - idx += 1; + if removal { + self.orderings.swap_remove(idx); + } else { + idx += 1; + } } } } @@ -768,12 +759,18 @@ pub fn add_offset_to_expr( // 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). -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()) +/// Trims `orderings[idx]` if some suffix of it overlaps with a prefix of +/// `orderings[pre_idx]`. Returns `true` if there is any overlap, `false` otherwise. +fn resolve_overlap(orderings: &mut [LexOrdering], idx: usize, pre_idx: usize) -> bool { + let length = orderings[idx].len(); + let other_length = orderings[pre_idx].len(); + for overlap in 1..=length.min(other_length) { + if orderings[idx][length - overlap..] == orderings[pre_idx][..overlap] { + orderings[idx].truncate(length - overlap); + return true; + } + } + false } /// A `EquivalenceProperties` object stores useful information related to a schema. @@ -1185,166 +1182,121 @@ impl EquivalenceProperties { self.eq_group.project_expr(projection_mapping, expr) } - /// Retrieves the leading orderings among all valid orderings. - /// - /// Leading ordering is the first non-constant ordering inside each - /// lexicographical ordering. - /// - /// # Returns - /// - /// A vector of `PhysicalSortExpr` representing the leading orderings. - fn get_leading_orderings(&self) -> Vec { - self.normalized_oeq_class() - .iter() - .flat_map(|ordering| { - // Get first non-constant sort_expr. - ordering - .iter() - .find(|sort_expr| !self.is_expr_constant(&sort_expr.expr)) - .cloned() - }) - .collect() - } - - /// Retrieves the ordered expressions after projections - /// - /// # Returns - /// - /// A tuple of `Vec`, - /// - first entry contains orderings that cannot accept further suffix orderings. - /// - second entry contains orderings that can accept further suffix orderings. - fn get_ordered_exprs_after_projection( - &self, - mapping: &ProjectionMapping, - ) -> (Vec, Vec) { - let leading_orderings = self.get_leading_orderings(); - let leading_ordering_exprs = leading_orderings - .iter() - .map(|sort_expr| sort_expr.expr.clone()) - .collect::>(); - - // Find which projection expressions have an order after projection. - let ordered_projections = mapping - .iter() - .flat_map(|(source, target)| { - let expr_ordering = self.get_expr_ordering(source.clone()); - if let SortProperties::Ordered(options) = expr_ordering.state { - Some(PhysicalSortExpr { - expr: target.clone(), - options, - }) - } else { - // expression is not ordered, - None + fn projected_orderings(&self, mapping: &ProjectionMapping) -> Vec { + let mut all_dependencies = vec![]; + for ordering in self.normalized_oeq_class().iter() { + let mut dependencies: Vec = vec![]; + for sort_expr in ordering { + let target_expr = self.eq_group.project_expr(mapping, &sort_expr.expr); + let target_sort_expr = target_expr.map(|expr| PhysicalSortExpr { + expr, + options: sort_expr.options, + }); + if PRINT_ON { + println!("sort_expr: {:?}", sort_expr); + println!("target_sort_expr: {:?}", target_sort_expr); } - }) - .collect::>(); - - // Split ordered projection expressions to the either complete or continuing ordering expressions - // - Complete ordering expressions means that, these orderings cannot accept suffix orderings after them. - // (Because they cannot guarantee how its arguments map during function, such as floor(a), round(a), etc.) - // This means that we cannot guarantee when floor(a) have a fixed value, a will also have a fixed value. - // Hence, ordering [x, a, b] shouldn't be projected as `[x, floor(a), b]` bu as `[x, floor(a)]` - // - Continuing ordering expressions means that, these orderings can accept suffix orderings after them. - let (mut new_orderings_continuing, mut new_orderings_complete): ( - Vec, - Vec, - ) = ordered_projections.into_iter().partition(|sort_expr| { - // If expr is one of the leading ordering exprs. This means that it is not a composite expression - // Hence its exactness doesn't depend on arguments. These sort expression can accept suffix (e.g in continuing mode). - // Others cannot accept suffix (e.g in completed mode) - // TODO: If we know that composite (complex) expression is 1-to-1 function. - // we can still continue iteration. If a is among leading orderings, exp(a) function - // can still continue. Even if it is not directly, among leading orderings. However, - // after floor(a) cannot continue iteration. - physical_exprs_contains(&leading_ordering_exprs, &sort_expr.expr) - }); - - // Project existing leading orderings. If leading ordering is `a+b` - // and mapping is `a as a_new`, `b as b_new`; Projected leading ordering - // would be `a_new+b_new`. - // Please note that: loop above where we iterate thorough all projections - // to understand ordering of the each projection expression cannot handles this case. - // Because neither `a_new` and `b_new` are themselves ordered. But their sum is ordered. - let projected_leading_orderings = leading_orderings - .into_iter() - .flat_map(|sort_expr| { - self.eq_group - .project_expr(mapping, &sort_expr.expr) - .map(|expr| PhysicalSortExpr { - expr, - options: sort_expr.options, - }) - }) - .collect::>(); - // Add projected leading orderings to the continuing orderings. - new_orderings_continuing.extend(projected_leading_orderings); - - // De-duplicate orderings - new_orderings_complete = deduplicate_sort_exprs(new_orderings_complete); - new_orderings_continuing = deduplicate_sort_exprs(new_orderings_continuing); - (new_orderings_complete, new_orderings_continuing) - } + let is_projected = target_sort_expr.is_some(); + if is_projected | any_projection_refers(mapping, &sort_expr.expr) { + let dependency_idx = if dependencies.is_empty() { + None + } else { + Some(all_dependencies.len() + dependencies.len() - 1) + }; + dependencies.push(Dependency { + source: sort_expr.clone(), + target: target_sort_expr, + dependency: dependency_idx, + }); + } + if !is_projected { + // TOD0: Iteration should stop when an expression cannot projected + // leaf dependency can be a referred expression + break; + } + } + all_dependencies.extend(dependencies); + } + if PRINT_ON { + for dependency in &all_dependencies { + println!("dependency: {:?}", dependency); + } + } - /// Projects `ordering` according to the given projection mapping. - /// If the resulting ordering is invalid after projection, returns `None`. - fn project_ordering( - mut self, - mapping: &ProjectionMapping, - ordering: LexOrderingRef, - ) -> Option> { - let mut completed_orderings: Vec = vec![]; - let mut continuing_orderings: Vec = vec![]; - for sort_expr in ordering.iter() { - let (new_orderings_complete, new_ordering_continuing) = - self.get_ordered_exprs_after_projection(mapping); - let projected_constants = self.projected_constants(mapping); - - // Update constants such that they treat left side of the lex ordering as constant during ordering discovery. - self = self.add_constants(std::iter::once(sort_expr.expr.clone())); - - let (relevant_orderings, non_relevant_orderings): ( - Vec, - Vec, - ) = if continuing_orderings.is_empty() { - (vec![vec![]], vec![vec![]]) - } else { - // Partition available orderings so that they orderings that can accept new - // suffix ordering and that cannot accept are seperated. - continuing_orderings.into_iter().partition(|ordering| { - is_ordering_constant(&projected_constants, ordering) + let mut completed_orderings = vec![]; + for (source, target) in mapping.iter() { + let relevant_indices = relevant_dependencies(&all_dependencies, source); + let relevant_prefixes = relevant_indices + .iter() + .map(|idx| { + construct_orderings1( + &all_dependencies, + all_dependencies[*idx].dependency, + ) }) - }; - - completed_orderings.extend(non_relevant_orderings); - let new_completed_orderings = if new_orderings_complete.is_empty() { - relevant_orderings.clone() - } else { - suffix_relevant_orderings( - relevant_orderings.clone(), - &new_orderings_complete, - ) - }; - completed_orderings.extend(new_completed_orderings); - - continuing_orderings = - suffix_relevant_orderings(relevant_orderings, &new_ordering_continuing); - - continuing_orderings = continuing_orderings - .into_iter() - .map(|ordering| self.eq_group.normalize_sort_exprs(&ordering)) - .collect(); - if continuing_orderings.is_empty() { - // Cannot continue iteration safely. Subsequent orderings cannot be calculated properly. - break; + .collect::>(); + assert_eq!(relevant_indices.len(), relevant_prefixes.len()); + let relevant_sources = relevant_indices + .iter() + .map(|idx| &all_dependencies[*idx].source) + .collect::>(); + let ordering = get_ordering_expr(source, &relevant_sources); + if PRINT_ON { + println!("relevant_sources: {:?}", relevant_sources); + println!("relevant_prefixes: {:?}", relevant_prefixes); + println!("ordering: {:?}", ordering); + println!("source: {:?}", source); } + if let SortProperties::Ordered(options) = ordering { + let sort_expr = PhysicalSortExpr { + expr: target.clone(), + options, + }; + let relevant_source_indices = + get_relevant_sources_indices(source, &relevant_sources); + if PRINT_ON { + println!("relevant_source_indices: {:?}", relevant_source_indices); + } + for indices in relevant_source_indices { + // TODO: Generate these as cross product not 0th index. + let prefix_to_use = indices + .iter() + .map(|idx| relevant_prefixes[*idx][0].clone()) + .collect::>(); + if PRINT_ON { + println!("indices: {:?}", indices); + println!("prefix_to_use: {:?}", prefix_to_use); + } + for elems in prefix_to_use.iter().permutations(prefix_to_use.len()) { + let mut ordering = + elems.into_iter().flatten().cloned().collect::>(); + ordering.push(sort_expr.clone()); + if PRINT_ON { + println!("new ordering: {:?}", ordering); + } + completed_orderings.push(ordering); + } + } + } + if PRINT_ON { + println!("relevant indices: {:?}", relevant_indices); + println!("completed_orderings: {:?}", completed_orderings); + } + } + // TODO: iterate only leaf dependencies. + for (idx, _dependency) in all_dependencies.iter().enumerate() { + let mut prefixes = construct_orderings1(&all_dependencies, Some(idx)); + if let Some(target_sort_expr) = &all_dependencies[idx].target { + prefixes + .iter_mut() + .for_each(|ordering| ordering.push(target_sort_expr.clone())); + } + completed_orderings.extend(prefixes); } - // Merge completed and continuing orderings. - let mut orderings = completed_orderings; - orderings.extend(continuing_orderings); - let oeq_class = OrderingEquivalenceClass::new(orderings); - orderings = oeq_class.orderings; - (!orderings.is_empty()).then_some(orderings) + completed_orderings + .into_iter() + .map(collapse_lex_ordering) + .collect() } /// Projects constants based on the provided `ProjectionMapping`. @@ -1393,13 +1345,7 @@ impl EquivalenceProperties { ) -> Self { let projected_constants = self.projected_constants(projection_mapping); let projected_eq_group = self.eq_group.project(projection_mapping); - - let projected_orderings = self - .oeq_class - .iter() - .filter_map(|order| self.clone().project_ordering(projection_mapping, order)) - .flatten() - .collect::>(); + let projected_orderings = self.projected_orderings(projection_mapping); Self { eq_group: projected_eq_group, oeq_class: OrderingEquivalenceClass::new(projected_orderings), @@ -1544,13 +1490,182 @@ fn is_expr_constant_util( } } -fn is_ordering_constant( - ignored_exprs: &[Arc], - ordering: &[PhysicalSortExpr], +#[derive(Debug, Clone)] +struct Dependency { + source: PhysicalSortExpr, + target: Option, + dependency: Option, +} + +fn expr_refers( + referring_expr: &Arc, + referred_expr: &Arc, ) -> bool { - ordering + if referring_expr.eq(referred_expr) { + true + } else { + referring_expr + .children() + .iter() + .any(|child| expr_refers(child, referred_expr)) + } +} + +fn relevant_dependencies( + dependencies: &[Dependency], + source: &Arc, +) -> Vec { + dependencies + .iter() + .enumerate() + .flat_map(|(idx, dependency)| { + if expr_refers(source, &dependency.source.expr) { + Some(idx) + } else { + None + } + }) + .collect() +} + +fn get_ordering_expr( + expr: &Arc, + relevant_column_orderings: &[&PhysicalSortExpr], +) -> SortProperties { + for column_order in relevant_column_orderings { + if expr.eq(&column_order.expr) { + return SortProperties::Ordered(column_order.options); + } + } + let child_states = expr + .children() + .iter() + .map(|child| get_ordering_expr(child, relevant_column_orderings)) + .collect::>(); + expr.get_ordering(&child_states) +} + +fn get_relevant_sources_indices( + expr: &Arc, + relevant_column_orderings: &[&PhysicalSortExpr], +) -> Vec> { + let mut result = vec![]; + for (idx, column_order) in relevant_column_orderings.iter().enumerate() { + if expr.eq(&column_order.expr) { + result.push(vec![idx]); + } + } + if !result.is_empty() { + return result; + } + let mut child_source_indices = expr + .children() + .iter() + .map(|child| get_relevant_sources_indices(child, relevant_column_orderings)) + .collect::>(); + child_source_indices.retain(|source_indices| !source_indices.is_empty()); + + if child_source_indices.is_empty() { + return result; + } + let bounds = child_source_indices .iter() - .all(|sort_expr| is_expr_constant_util(ignored_exprs, &sort_expr.expr)) + .map(|child_source| child_source.len()) + .collect::>(); + let mut gen = Generator::new(bounds); + while let Some(indices) = gen.next() { + let mut vals = vec![]; + izip!(indices.iter(), child_source_indices.iter()).for_each( + |(idx, child_source)| { + vals.extend(child_source[*idx].clone()); + }, + ); + result.push(vals); + } + + result +} + +struct Generator { + lengths: Vec, + idx: usize, + seed: Vec, + emitted: bool, +} + +impl Generator { + fn new(lengths: Vec) -> Self { + let idx = lengths.len() - 1; + let seed = vec![0; lengths.len()]; + Self { + lengths, + idx, + seed, + emitted: false, + } + } + + fn next(&mut self) -> Option> { + if !self.emitted { + self.emitted = true; + return Some(self.seed.clone()); + } + let mut idx_to_increment = self.idx; + loop { + if self.increment_at_index(idx_to_increment) { + return Some(self.seed.clone()); + } else if idx_to_increment > 0 { + // Reset + self.seed[idx_to_increment] = 0; + idx_to_increment -= 1; + } else { + return None; + } + } + } + + fn increment_at_index(&mut self, idx: usize) -> bool { + if self.seed[idx] + 1 == self.lengths[idx] { + false + } else { + self.seed[idx] += 1; + true + } + } +} + +fn construct_orderings1( + dependencies: &[Dependency], + relevant_idx: Option, +) -> Vec> { + if let Some(relevant_idx) = relevant_idx { + let mut result = + construct_orderings1(dependencies, dependencies[relevant_idx].dependency); + if let Some(target_sort_expr) = &dependencies[relevant_idx].target { + result + .iter_mut() + .for_each(|ordering| ordering.push(target_sort_expr.clone())); + result + } else { + vec![] + } + } else { + // Empty Ordering. + vec![vec![]] + } +} + +// TODO: Consider equivalences also. +fn any_projection_refers( + mapping: &ProjectionMapping, + expr: &Arc, +) -> bool { + for (source, _target) in mapping.iter() { + if expr_refers(source, expr) { + return true; + } + } + false } /// Calculate ordering equivalence properties for the given join operation. @@ -1689,22 +1804,6 @@ fn update_ordering( } } -/// Suffix each new_ordering to the end of the each relevant ordering. -fn suffix_relevant_orderings( - relevant_orderings: Vec, - new_orderings: &[PhysicalSortExpr], -) -> Vec { - let mut result = vec![]; - for relevant_ordering in &relevant_orderings { - for new_ordering in new_orderings { - let mut ordering = relevant_ordering.clone(); - ordering.push(new_ordering.clone()); - result.push(ordering); - } - } - result -} - /// This function constructs a duplicate-free `Vec` by filtering out /// duplicate entries that have same `PhysicalSortExpr` inside. pub fn deduplicate_sort_exprs(input: Vec) -> Vec { @@ -4034,7 +4133,6 @@ mod tests { // [a_new ASC, d_new ASC] vec![("a_new", option_asc), ("d_new", option_asc)], // [a_new ASC, b+d ASC] - // TODO: In below case lexicographical argument ordering may be unnecessary for 1-to-1 functions. vec![("a_new", option_asc), ("b+d", option_asc)], ], ), @@ -4086,7 +4184,7 @@ mod tests { ("d_new", option_asc), ("b_new", option_asc), ], - // [c_new ASC] + // [c_new ASC], vec![("c_new", option_asc)], ], ), @@ -4205,17 +4303,12 @@ mod tests { ("a+b", option_asc), ("c_new", option_asc), ], - // [a_new ASC, a+b ASC, b_new ASC] - vec![ - ("a_new", option_asc), - ("a+b", option_asc), - ("b_new", option_asc), - ], ], ), ]; - for (orderings, proj_exprs, expected) in test_cases { + for (idx, (orderings, proj_exprs, expected)) in test_cases.into_iter().enumerate() + { let mut eq_properties = EquivalenceProperties::new(schema.clone()); let orderings = convert_to_orderings(&orderings); @@ -4245,8 +4338,8 @@ mod tests { let orderings = projected_eq.oeq_class(); let err_msg = format!( - "actual: {:?}, expected: {:?}, projection_mapping: {:?}", - orderings.orderings, expected, projection_mapping + "test_idx: {:?}, actual: {:?}, expected: {:?}, projection_mapping: {:?}", + idx, orderings.orderings, expected, projection_mapping ); assert_eq!(orderings.len(), expected.len(), "{}", err_msg); @@ -4415,23 +4508,41 @@ mod tests { vec![(col_c_new, option_asc), (col_b_new, option_asc)], ], ), + // ---------- TEST CASE 7 ------------ + ( + // orderings + vec![ + // [a+b ASC, c ASC] + vec![(&a_plus_b, option_asc), (col_c, option_asc)], + ], + // expected + vec![ + // [a+b ASC, round(c) ASC, c_new ASC] + vec![ + (&a_new_plus_b_new, option_asc), + (&col_round_c_res, option_asc), + ], + // [a+b ASC, c_new ASC] + vec![(&a_new_plus_b_new, option_asc), (col_c_new, option_asc)], + ], + ), ]; - for (orderings, expected) in test_cases { + for (idx, (orderings, expected)) in test_cases.iter().enumerate() { let mut eq_properties = EquivalenceProperties::new(schema.clone()); - let orderings = convert_to_orderings(&orderings); + let orderings = convert_to_orderings(orderings); eq_properties.add_new_orderings(orderings); - let expected = convert_to_orderings(&expected); + let expected = convert_to_orderings(expected); let projected_eq = eq_properties.project(&projection_mapping, output_schema.clone()); let orderings = projected_eq.oeq_class(); let err_msg = format!( - "actual: {:?}, expected: {:?}, projection_mapping: {:?}", - orderings.orderings, expected, projection_mapping + "test idx: {:?}, actual: {:?}, expected: {:?}, projection_mapping: {:?}", + idx, orderings.orderings, expected, projection_mapping ); assert_eq!(orderings.len(), expected.len(), "{}", err_msg); @@ -4479,7 +4590,7 @@ mod tests { let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &schema)?; let output_schema = output_schema(&projection_mapping, &schema)?; - let _col_a_plus_b_new = &col("a+b", &output_schema)?; + let col_a_plus_b_new = &col("a+b", &output_schema)?; let col_c_new = &col("c_new", &output_schema)?; let col_d_new = &col("d_new", &output_schema)?; @@ -4495,14 +4606,18 @@ mod tests { ], // expected vec![ - // TODO: Ordering below is also valid. We should be able to understand it is valid. - // For now, we cannot understand it is valid. - // // [d_new ASC, c_new ASC, a+b ASC] - // vec![(col_d_new, option_asc), (col_c_new, option_asc), (_col_a_plus_b_new, option_asc)], - // [d_new ASC] - vec![(col_d_new, option_asc)], - // [c_new ASC] - vec![(col_c_new, option_asc)], + // [d_new ASC, c_new ASC, a+b ASC] + vec![ + (col_d_new, option_asc), + (col_c_new, option_asc), + (col_a_plus_b_new, option_asc), + ], + // [c_new ASC, d_new ASC, a+b ASC] + vec![ + (col_c_new, option_asc), + (col_d_new, option_asc), + (col_a_plus_b_new, option_asc), + ], ], ), ]; @@ -4605,6 +4720,101 @@ mod tests { Ok(()) } + #[test] + fn ordering_satisfy_after_projection_random() -> Result<()> { + const N_RANDOM_SCHEMA: usize = 20; + 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 { + println!("{}/{}", seed + 1, N_RANDOM_SCHEMA); + // Create a random schema with random properties + 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_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; + // Floor(a) + let floor_a = create_physical_expr( + &BuiltinScalarFunction::Floor, + &[col("a", &test_schema)?], + &test_schema, + &ExecutionProps::default(), + )?; + // a + b + let a_plus_b = Arc::new(BinaryExpr::new( + col("a", &test_schema)?, + Operator::Plus, + col("b", &test_schema)?, + )) as Arc; + let proj_exprs = vec![ + (col("a", &test_schema)?, "a_new"), + (col("b", &test_schema)?, "b_new"), + (col("c", &test_schema)?, "c_new"), + (col("d", &test_schema)?, "d_new"), + (col("e", &test_schema)?, "e_new"), + (col("f", &test_schema)?, "f_new"), + (floor_a, "floor(a)"), + (a_plus_b, "a+b"), + ]; + + for n_req in 0..=proj_exprs.len() { + for proj_exprs in proj_exprs.iter().combinations(n_req) { + let proj_exprs = proj_exprs + .into_iter() + .map(|(expr, name)| (expr.clone(), name.to_string())) + .collect::>(); + let (projected_batch, projected_eq) = apply_projection( + proj_exprs.clone(), + &table_data_with_properties, + &eq_properties, + )?; + + let projection_mapping = + ProjectionMapping::try_new(&proj_exprs, &test_schema)?; + + let projected_exprs = projection_mapping + .iter() + .map(|(_source, target)| target.clone()) + .collect::>(); + + for n_req in 0..=projected_exprs.len() { + for exprs in projected_exprs.iter().combinations(n_req) { + let requirement = exprs + .into_iter() + .map(|expr| PhysicalSortExpr { + expr: expr.clone(), + options: SORT_OPTIONS, + }) + .collect::>(); + let expected = is_table_same_after_sort( + requirement.clone(), + projected_batch.clone(), + )?; + let err_msg = format!( + "Error in test case requirement:{:?}, expected: {:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}, projected_eq.oeq_class: {:?}, projected_eq.eq_group: {:?}, projected_eq.constants: {:?}, projection_mapping: {:?}", + requirement, expected, eq_properties.oeq_class, eq_properties.eq_group, eq_properties.constants, projected_eq.oeq_class, projected_eq.eq_group, projected_eq.constants, projection_mapping + ); + // Check whether ordering_satisfy API result and + // experimental result matches. + assert_eq!( + projected_eq.ordering_satisfy(&requirement), + expected, + "{}", + err_msg + ); + } + } + } + } + } + + Ok(()) + } + #[test] fn test_expr_consists_of_constants() -> Result<()> { let schema = Arc::new(Schema::new(vec![ @@ -4632,4 +4842,13 @@ mod tests { assert!(is_expr_constant_util(&constants, &expr)); Ok(()) } + + #[test] + fn test_generator() -> Result<()> { + let mut gen = Generator::new(vec![2, 2, 2]); + while let Some(elem) = gen.next() { + println!("elem:{:?}", elem); + } + Ok(()) + } } From e429e89e7fa57f2ef6c1298621c39e6201c221f2 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 21 Nov 2023 18:29:30 +0300 Subject: [PATCH 37/44] Change ordering projection implementation --- datafusion/physical-expr/src/equivalence.rs | 739 +++++++++++++------- 1 file changed, 495 insertions(+), 244 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 48bd6b327028..5da385c83a02 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::{HashMap, HashSet}; use std::hash::Hash; use std::sync::Arc; @@ -32,9 +33,7 @@ use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{JoinSide, JoinType, Result}; use indexmap::IndexSet; -use itertools::{izip, Itertools}; - -const PRINT_ON: bool = false; +use itertools::Itertools; /// 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 @@ -1182,121 +1181,188 @@ impl EquivalenceProperties { self.eq_group.project_expr(projection_mapping, expr) } - fn projected_orderings(&self, mapping: &ProjectionMapping) -> Vec { - let mut all_dependencies = vec![]; + /// Constructs a dependency map based on existing orderings referred to in the projection. + /// + /// This function analyzes the orderings in the normalized order-equivalence class and + /// builds a dependency map. The dependency map captures relationships between expressions + /// within the orderings, helping to identify dependencies and construct valid projected + /// orderings during projection operations. + /// + /// # Parameters + /// + /// - `mapping`: A reference to the `ProjectionMapping` that defines the relationship + /// between source and target expressions. + /// + /// # Returns + /// + /// A `HashMap` representing the dependency map, where each + /// [`DependencyNode`] contains dependencies for the key [`PhysicalSortExpr`]. + /// + /// As an example: Assume mapping is [a->a_new, b->b_new, c->c_new], Ordering [a ASC, b ASC, c DESC] will be transformed to the + /// a ASC: Node{a_new ASC, HashSet{}} + /// b ASC: Node{b_new ASC, HashSet{a ASC}} + /// c DESC: Node{c_new DESC, HashSet{b ASC}}. + fn construct_dependency_map( + &self, + mapping: &ProjectionMapping, + ) -> HashMap { + // Construct dependency map of the existing orderings that are referred in the projection. + let mut dependency_map = HashMap::new(); for ordering in self.normalized_oeq_class().iter() { - let mut dependencies: Vec = vec![]; - for sort_expr in ordering { - let target_expr = self.eq_group.project_expr(mapping, &sort_expr.expr); + for (idx, sort_expr) in ordering.iter().enumerate() { + // Previous ordering is dependency. For leading ordering (e.g first sort_expr inside ordering) + // there is no dependency. + // let dependency = (idx > 0).then_some(&ordering[idx - 1]); + let dependency = if idx > 0 { + Some(&ordering[idx - 1]) + } else { + None + }; + let target_expr = self.project_expr(&sort_expr.expr, mapping); let target_sort_expr = target_expr.map(|expr| PhysicalSortExpr { expr, options: sort_expr.options, }); - if PRINT_ON { - println!("sort_expr: {:?}", sort_expr); - println!("target_sort_expr: {:?}", target_sort_expr); - } let is_projected = target_sort_expr.is_some(); if is_projected | any_projection_refers(mapping, &sort_expr.expr) { - let dependency_idx = if dependencies.is_empty() { - None - } else { - Some(all_dependencies.len() + dependencies.len() - 1) - }; - dependencies.push(Dependency { - source: sort_expr.clone(), - target: target_sort_expr, - dependency: dependency_idx, - }); + // Add sort_exprs that + // - can be projected + // - referred by any of the projection expressions + // to the dependency map. + dependency_map + .entry(sort_expr.clone()) + .or_insert_with(|| DependencyNode { + target_sort_expr: target_sort_expr.clone(), + dependencies: HashSet::new(), + }) + .insert(dependency); } if !is_projected { - // TOD0: Iteration should stop when an expression cannot projected - // leaf dependency can be a referred expression + // If expression cannot be projected + // Stop constructing dependency. Remaining dependencies will not be valid after projection. break; } } - all_dependencies.extend(dependencies); - } - if PRINT_ON { - for dependency in &all_dependencies { - println!("dependency: {:?}", dependency); - } } + dependency_map + } + + /// Returns a new `ProjectionMapping` where source expressions are normalized. + /// + /// The normalization ensures that source expressions are transformed into a consistent + /// representation. This is beneficial for algorithms that rely on exact equalities, + /// as it allows for more precise and reliable comparisons. + /// + /// # Parameters + /// + /// - `mapping`: A reference to the original `ProjectionMapping` to be normalized. + /// + /// # Returns + /// + /// A new `ProjectionMapping` with normalized source expressions. + fn normalized_mapping(&self, mapping: &ProjectionMapping) -> ProjectionMapping { + // Construct the mapping where source expressions are normalized. In this way + // In the algorithms below we can work on exact equalities + let new_inner = mapping + .iter() + .map(|(source, target)| { + let normalized_source = self.eq_group.normalize_expr(source.clone()); + (normalized_source, target.clone()) + }) + .collect::>(); + ProjectionMapping { inner: new_inner } + } + + /// Computes projected orderings based on a given projection mapping. + /// + /// This function takes a `ProjectionMapping` and computes the possible orderings + /// for the projected expressions. It considers dependencies between expressions + /// and generates valid orderings according to the specified sort properties. + /// + /// # Parameters + /// + /// - `mapping`: A reference to the `ProjectionMapping` that defines the relationship + /// between source and target expressions. + /// + /// # Returns + /// + /// A vector of `LexOrdering` containing all of the valid orderings after projection. + fn projected_orderings(&self, mapping: &ProjectionMapping) -> Vec { + let mapping = self.normalized_mapping(mapping); - let mut completed_orderings = vec![]; + // Get dependency map for existing orderings. + let dependency_map = self.construct_dependency_map(&mapping); + + let mut orderings = vec![]; for (source, target) in mapping.iter() { - let relevant_indices = relevant_dependencies(&all_dependencies, source); - let relevant_prefixes = relevant_indices + let relevant_deps = referred_dependencies(&dependency_map, source); + let ordering = get_expr_ordering(source, &relevant_deps); + let options = if let SortProperties::Ordered(options) = ordering { + options + } else { + // expression is not ordered. Try next projection + continue; + }; + let sort_expr = PhysicalSortExpr { + expr: target.clone(), + options, + }; + + // Construct all of the valid prefix orderings for each of the expressions, that is referred in the projection expression. + let relevant_prefixes = relevant_deps .iter() - .map(|idx| { - construct_orderings1( - &all_dependencies, - all_dependencies[*idx].dependency, - ) + .flat_map(|dep| { + let alternative_prefixes = + construct_prefix_orderings(dep, &dependency_map); + (!alternative_prefixes.is_empty()).then_some(alternative_prefixes) }) .collect::>(); - assert_eq!(relevant_indices.len(), relevant_prefixes.len()); - let relevant_sources = relevant_indices - .iter() - .map(|idx| &all_dependencies[*idx].source) - .collect::>(); - let ordering = get_ordering_expr(source, &relevant_sources); - if PRINT_ON { - println!("relevant_sources: {:?}", relevant_sources); - println!("relevant_prefixes: {:?}", relevant_prefixes); - println!("ordering: {:?}", ordering); - println!("source: {:?}", source); + + // No dependency, it is a leading ordering + if relevant_prefixes.is_empty() { + orderings.push(vec![sort_expr.clone()]); } - if let SortProperties::Ordered(options) = ordering { - let sort_expr = PhysicalSortExpr { - expr: target.clone(), - options, - }; - let relevant_source_indices = - get_relevant_sources_indices(source, &relevant_sources); - if PRINT_ON { - println!("relevant_source_indices: {:?}", relevant_source_indices); - } - for indices in relevant_source_indices { - // TODO: Generate these as cross product not 0th index. - let prefix_to_use = indices - .iter() - .map(|idx| relevant_prefixes[*idx][0].clone()) - .collect::>(); - if PRINT_ON { - println!("indices: {:?}", indices); - println!("prefix_to_use: {:?}", prefix_to_use); - } - for elems in prefix_to_use.iter().permutations(prefix_to_use.len()) { - let mut ordering = - elems.into_iter().flatten().cloned().collect::>(); - ordering.push(sort_expr.clone()); - if PRINT_ON { - println!("new ordering: {:?}", ordering); - } - completed_orderings.push(ordering); - } + + // Generate all possible orderings where dependencies are satisfied for the current projection expression. + // If expression is a+b ASC, and dependency for a ASC is [c ASC], dependency for b ASC is [d DESC]. + // Then we generate [c ASC, d DESC, a+b ASC], [d DESC, c ASC, a+b ASC]. + for prefix_orderings in + relevant_prefixes.into_iter().multi_cartesian_product() + { + for prefixes in + prefix_orderings.iter().permutations(prefix_orderings.len()) + { + let ordering = prefixes + .into_iter() + .flatten() + .chain(std::iter::once(&sort_expr)) + .cloned() + .collect(); + orderings.push(ordering); } } - if PRINT_ON { - println!("relevant indices: {:?}", relevant_indices); - println!("completed_orderings: {:?}", completed_orderings); - } } - // TODO: iterate only leaf dependencies. - for (idx, _dependency) in all_dependencies.iter().enumerate() { - let mut prefixes = construct_orderings1(&all_dependencies, Some(idx)); - if let Some(target_sort_expr) = &all_dependencies[idx].target { - prefixes - .iter_mut() - .for_each(|ordering| ordering.push(target_sort_expr.clone())); + + // Add valid projected orderings. + for (sort_expr, node) in dependency_map.iter() { + let mut prefixes = construct_prefix_orderings(sort_expr, &dependency_map); + if prefixes.is_empty() { + // If prefix is empty, there is no dependency. + // `node.target_sort_expr` is itself leading ordering, add it to the orderings. + if let Some(target) = &node.target_sort_expr { + orderings.push(vec![target.clone()]); + } } - completed_orderings.extend(prefixes); + // Append current ordering on top its dependencies + prefixes.iter_mut().for_each(|ordering| { + if let Some(target) = &node.target_sort_expr { + ordering.push(target.clone()) + } + }); + orderings.extend(prefixes); } - completed_orderings - .into_iter() - .map(collapse_lex_ordering) - .collect() + + orderings.into_iter().map(collapse_lex_ordering).collect() } /// Projects constants based on the provided `ProjectionMapping`. @@ -1490,13 +1556,21 @@ fn is_expr_constant_util( } } -#[derive(Debug, Clone)] -struct Dependency { - source: PhysicalSortExpr, - target: Option, - dependency: Option, -} - +/// Checks if a referring expression refers to a given referred expression. +/// +/// This function examines whether a referring expression directly refers to a +/// given referred expression or if any of its children in the expression tree +/// refer to the specified expression. +/// +/// # Parameters +/// +/// - `referring_expr`: A reference to the referring expression (`Arc`). +/// - `referred_expr`: A reference to the referred expression (`Arc`) +/// +/// # Returns +/// +/// A boolean value indicating whether `referring_expr` refers (needs it to evaluate its result) +/// `referred_expr` or not. fn expr_refers( referring_expr: &Arc, referred_expr: &Arc, @@ -1511,16 +1585,32 @@ fn expr_refers( } } -fn relevant_dependencies( - dependencies: &[Dependency], +/// Collests referred dependencies for a given source expression. +/// +/// This function analyzes the dependency map to collect referred dependencies for +/// a given source expression. +/// +/// # Parameters +/// +/// - `dependency_map`: A reference to the `HashMap` representing +/// the dependency map, where each expression is associated with a `Node`. +/// - `source`: A reference to the source expression (`Arc`) for which +/// relevant dependencies need to be identified. +/// +/// # Returns +/// +/// A `HashSet` containing relevant dependencies for the given source +/// expression. These dependencies are expressions that are referred to by the source +/// expression based on the provided dependency map. +fn referred_dependencies( + dependency_map: &HashMap, source: &Arc, -) -> Vec { - dependencies +) -> HashSet { + dependency_map .iter() - .enumerate() - .flat_map(|(idx, dependency)| { - if expr_refers(source, &dependency.source.expr) { - Some(idx) + .filter_map(|(sort_expr, _node)| { + if expr_refers(source, &sort_expr.expr) { + Some(sort_expr.clone()) } else { None } @@ -1528,144 +1618,157 @@ fn relevant_dependencies( .collect() } -fn get_ordering_expr( - expr: &Arc, - relevant_column_orderings: &[&PhysicalSortExpr], -) -> SortProperties { - for column_order in relevant_column_orderings { - if expr.eq(&column_order.expr) { - return SortProperties::Ordered(column_order.options); - } +/// Constructs Lexicographical Orderings based on the given relevant sort expression and dependency map. +/// +/// This function recursively analyzes the dependencies of the provided relevant sort expression +/// within the given dependency map. It constructs lexicographical orderings that include the +/// relevant sort expression and its dependencies. +/// +/// # Parameters +/// +/// - `referred_sort_expr`: A reference to the relevant sort expression (`PhysicalSortExpr`) +/// for which lexicographical orderings need to be constructed, that satisfying its dependencies. +/// - `dependency_map`: A reference to the `HashMap` that contains dependencies +/// for different `PhysicalSortExpr`s. +/// +/// # Returns +/// +/// A vector of lexicographical orderings (`Vec`) based on the provided relevant +/// sort expression and its dependencies. +fn construct_orderings( + referred_sort_expr: &PhysicalSortExpr, + dependency_map: &HashMap, +) -> Vec { + // We are sure that `referred_sort_expr` is inside `dependency_map`. + let val = &dependency_map[referred_sort_expr]; + // Since these nodes are intermediate nodes. We are sure `val.target_sort_expr` is `Some(_)`; + let target_sort_expr = val.target_sort_expr.clone().unwrap(); + if val.dependencies.is_empty() { + vec![vec![target_sort_expr]] + } else { + val.dependencies + .iter() + .flat_map(|dep| { + let mut orderings = construct_orderings(dep, dependency_map); + orderings + .iter_mut() + .for_each(|ordering| ordering.push(target_sort_expr.clone())); + orderings + }) + .collect::>() } - let child_states = expr - .children() - .iter() - .map(|child| get_ordering_expr(child, relevant_column_orderings)) - .collect::>(); - expr.get_ordering(&child_states) } -fn get_relevant_sources_indices( - expr: &Arc, - relevant_column_orderings: &[&PhysicalSortExpr], -) -> Vec> { - let mut result = vec![]; - for (idx, column_order) in relevant_column_orderings.iter().enumerate() { - if expr.eq(&column_order.expr) { - result.push(vec![idx]); - } - } - if !result.is_empty() { - return result; - } - let mut child_source_indices = expr - .children() - .iter() - .map(|child| get_relevant_sources_indices(child, relevant_column_orderings)) - .collect::>(); - child_source_indices.retain(|source_indices| !source_indices.is_empty()); - - if child_source_indices.is_empty() { - return result; - } - let bounds = child_source_indices +/// Constructs prefix orderings (dependent orderings) based on the given relevant sort expression and dependency map. +/// +/// This function retrieves the dependencies of the provided relevant sort expression from the +/// given dependency map. It then constructs prefix orderings by recursively analyzing the +/// dependencies and including them in the orderings. +/// +/// # Parameters +/// +/// - `relevant_sort_expr`: A reference to the relevant sort expression (`PhysicalSortExpr`) +/// for which prefix orderings need to be constructed. +/// - `dependency_map`: A reference to the `HashMap` that contains dependencies +/// for different `PhysicalSortExpr`s. +/// +/// # Returns +/// +/// A vector of prefix orderings (`Vec`) based on the provided relevant sort +/// expression and its dependencies. +fn construct_prefix_orderings( + relevant_sort_expr: &PhysicalSortExpr, + dependency_map: &HashMap, +) -> Vec { + dependency_map[relevant_sort_expr] + .dependencies .iter() - .map(|child_source| child_source.len()) - .collect::>(); - let mut gen = Generator::new(bounds); - while let Some(indices) = gen.next() { - let mut vals = vec![]; - izip!(indices.iter(), child_source_indices.iter()).for_each( - |(idx, child_source)| { - vals.extend(child_source[*idx].clone()); - }, - ); - result.push(vals); - } - - result -} - -struct Generator { - lengths: Vec, - idx: usize, - seed: Vec, - emitted: bool, -} - -impl Generator { - fn new(lengths: Vec) -> Self { - let idx = lengths.len() - 1; - let seed = vec![0; lengths.len()]; - Self { - lengths, - idx, - seed, - emitted: false, - } - } - - fn next(&mut self) -> Option> { - if !self.emitted { - self.emitted = true; - return Some(self.seed.clone()); - } - let mut idx_to_increment = self.idx; - loop { - if self.increment_at_index(idx_to_increment) { - return Some(self.seed.clone()); - } else if idx_to_increment > 0 { - // Reset - self.seed[idx_to_increment] = 0; - idx_to_increment -= 1; - } else { - return None; - } - } - } - - fn increment_at_index(&mut self, idx: usize) -> bool { - if self.seed[idx] + 1 == self.lengths[idx] { - false - } else { - self.seed[idx] += 1; - true - } - } + .flat_map(|dep| construct_orderings(dep, dependency_map)) + .collect() } -fn construct_orderings1( - dependencies: &[Dependency], - relevant_idx: Option, -) -> Vec> { - if let Some(relevant_idx) = relevant_idx { - let mut result = - construct_orderings1(dependencies, dependencies[relevant_idx].dependency); - if let Some(target_sort_expr) = &dependencies[relevant_idx].target { - result - .iter_mut() - .for_each(|ordering| ordering.push(target_sort_expr.clone())); - result - } else { - vec![] - } +/// Retrieves the ordering properties for a given expression based on sort expressions it refers. +/// +/// This function examines the provided expression and sort expressions it refers to determine +/// the ordering properties of the expression. +/// +/// # Parameters +/// +/// - `expr`: A reference to the source expression (`Arc`) for which ordering +/// properties need to be determined. +/// - `referred_sort_exprs`: A reference to a `HashSet` containing sort +/// expressions referred by the `expr`. +/// +/// # Returns +/// +/// A `SortProperties` enum indicating the ordering information of the given expression: +fn get_expr_ordering( + expr: &Arc, + referred_sort_exprs: &HashSet, +) -> SortProperties { + if let Some(column_order) = referred_sort_exprs + .iter() + .find(|&order| expr.eq(&order.expr)) + { + // If exact match is found, return its ordering. + SortProperties::Ordered(column_order.options) } else { - // Empty Ordering. - vec![vec![]] + // Find orderings of its children + let child_states = expr + .children() + .iter() + .map(|child| get_expr_ordering(child, referred_sort_exprs)) + .collect::>(); + // Calculate expression ordering using ordering of its children. + expr.get_ordering(&child_states) } } -// TODO: Consider equivalences also. +/// Checks if any expression in the projection mapping refers to a given source expression. +/// +/// # Parameters +/// +/// - `mapping`: A reference to the `ProjectionMapping` that defines the relationship between +/// source and target expressions. +/// - `expr`: A reference to the `Arc` for which we check whether any projection +/// source expression refers to it.. +/// +/// # Returns +/// +/// A `bool` flag, that indicates, whether any projection source refers `expr` or not. fn any_projection_refers( mapping: &ProjectionMapping, expr: &Arc, ) -> bool { - for (source, _target) in mapping.iter() { - if expr_refers(source, expr) { - return true; + mapping + .iter() + .any(|(source, _target)| expr_refers(source, expr)) +} + +/// Represents a node in the dependency map used for constructing projected orderings. +/// +/// A `DependencyNode` contains information about a particular sort expression, including +/// its target sort expression and a set of dependencies on other sort expressions. +/// +/// # Fields +/// +/// - `target_sort_expr`: An optional `PhysicalSortExpr` representing the target sort expression +/// associated with the node. It is `None` if the sort expression cannot be projected. +/// - `dependencies`: A `HashSet` containing dependencies on other sort expressions +/// that are referred to by the target sort expression. +#[derive(Debug, Clone, PartialEq, Eq)] +struct DependencyNode { + target_sort_expr: Option, + dependencies: HashSet, +} + +impl DependencyNode { + // Insert dependency to the state, (if there is any dependency) + fn insert(&mut self, dependency: Option<&PhysicalSortExpr>) { + if let Some(dep) = dependency { + self.dependencies.insert(dep.clone()); } } - false } /// Calculate ordering equivalence properties for the given join operation. @@ -3957,12 +4060,14 @@ mod tests { Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), Field::new("d", DataType::Int32, true), + Field::new("e", DataType::Int32, true), Field::new("ts", DataType::Timestamp(TimeUnit::Nanosecond, None), true), ])); 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_ts = &col("ts", &schema)?; let interval = Arc::new(Literal::new(ScalarValue::IntervalDayTime(Some(2)))) as Arc; @@ -3982,6 +4087,11 @@ mod tests { Operator::Plus, col_d.clone(), )) as Arc; + let b_plus_e = Arc::new(BinaryExpr::new( + col_b.clone(), + Operator::Plus, + col_e.clone(), + )) as Arc; let c_plus_d = Arc::new(BinaryExpr::new( col_c.clone(), Operator::Plus, @@ -4305,8 +4415,106 @@ mod tests { ], ], ), + // ------- TEST CASE 14 ---------- + ( + // orderings + vec![ + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + // [c ASC, b ASC] + vec![(col_c, option_asc), (col_b, option_asc)], + // [d ASC, e ASC] + vec![(col_d, option_asc), (col_e, option_asc)], + ], + // proj exprs + vec![ + (col_c, "c_new".to_string()), + (col_d, "d_new".to_string()), + (col_a, "a_new".to_string()), + (&b_plus_e, "b+e".to_string()), + ], + // expected, This expected may be missing + vec![ + // [a_new ASC, d_new ASC, b+e ASC] + vec![ + ("a_new", option_asc), + ("d_new", option_asc), + ("b+e", option_asc), + ], + // [d_new ASC, a_new ASC, b+e ASC] + vec![ + ("d_new", option_asc), + ("a_new", option_asc), + ("b+e", option_asc), + ], + // [c_new ASC, d_new ASC, b+e ASC] + vec![ + ("c_new", option_asc), + ("d_new", option_asc), + ("b+e", option_asc), + ], + // [d_new ASC, c_new ASC, b+e ASC] + vec![ + ("d_new", option_asc), + ("c_new", option_asc), + ("b+e", option_asc), + ], + ], + ), + // ------- TEST CASE 15 ---------- + ( + // orderings + vec![ + // [a ASC, c ASC, b ASC] + vec![ + (col_a, option_asc), + (col_c, option_asc), + (&col_b, option_asc), + ], + ], + // proj exprs + vec![ + (col_c, "c_new".to_string()), + (col_a, "a_new".to_string()), + (&a_plus_b, "a+b".to_string()), + ], + // expected, This expected may be missing + vec![ + // [a_new ASC, d_new ASC, b+e ASC] + vec![ + ("a_new", option_asc), + ("c_new", option_asc), + ("a+b", option_asc), + ], + ], + ), ]; + // let test_cases = vec![ + // // ---------- TEST CASE 3 ------------ + // ( + // // orderings + // vec![ + // // [ts ASC] + // vec![(col_ts, option_asc)], + // ], + // // projection exprs + // vec![ + // (col_b, "b_new".to_string()), + // (col_a, "a_new".to_string()), + // (col_ts, "ts_new".to_string()), + // (date_bin_func, "date_bin_res".to_string()), + // ], + // // expected + // vec![ + // // [date_bin_res ASC] + // vec![("date_bin_res", option_asc)], + // // [ts_new ASC] + // vec![("ts_new", option_asc)], + // ], + // ), + // ]; + for (idx, (orderings, proj_exprs, expected)) in test_cases.into_iter().enumerate() { let mut eq_properties = EquivalenceProperties::new(schema.clone()); @@ -4561,12 +4769,15 @@ mod tests { Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), Field::new("d", DataType::Int32, true), - Field::new("ts", DataType::Timestamp(TimeUnit::Nanosecond, None), true), + Field::new("e", DataType::Int32, true), + Field::new("f", DataType::Int32, true), ])); 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 a_plus_b = Arc::new(BinaryExpr::new( col_a.clone(), Operator::Plus, @@ -4604,6 +4815,35 @@ mod tests { // [c ASC, a ASC] vec![(col_c, option_asc), (col_a, option_asc)], ], + // equal conditions + vec![], + // expected + vec![ + // [d_new ASC, c_new ASC, a+b ASC] + vec![ + (col_d_new, option_asc), + (col_c_new, option_asc), + (col_a_plus_b_new, option_asc), + ], + // [c_new ASC, d_new ASC, a+b ASC] + vec![ + (col_c_new, option_asc), + (col_d_new, option_asc), + (col_a_plus_b_new, option_asc), + ], + ], + ), + // ---------- TEST CASE 2 ------------ + ( + // orderings + vec![ + // [d ASC, b ASC] + vec![(col_d, option_asc), (col_b, option_asc)], + // [c ASC, e ASC], Please note that a=e + vec![(col_c, option_asc), (col_e, option_asc)], + ], + // equal conditions + vec![(col_e, col_a)], // expected vec![ // [d_new ASC, c_new ASC, a+b ASC] @@ -4620,10 +4860,31 @@ mod tests { ], ], ), + // ---------- TEST CASE 3 ------------ + ( + // orderings + vec![ + // [d ASC, b ASC] + vec![(col_d, option_asc), (col_b, option_asc)], + // [c ASC, e ASC], Please note that a=f + vec![(col_c, option_asc), (col_e, option_asc)], + ], + // equal conditions + vec![(col_a, col_f)], + // expected + vec![ + // [d_new ASC] + vec![(col_d_new, option_asc)], + // [c_new ASC] + vec![(col_c_new, option_asc)], + ], + ), ]; - - for (orderings, expected) in test_cases { + for (orderings, equal_columns, expected) in test_cases { let mut eq_properties = EquivalenceProperties::new(schema.clone()); + for (lhs, rhs) in equal_columns { + eq_properties.add_equal_conditions(lhs, rhs); + } let orderings = convert_to_orderings(&orderings); eq_properties.add_new_orderings(orderings); @@ -4731,7 +4992,6 @@ mod tests { }; for seed in 0..N_RANDOM_SCHEMA { - println!("{}/{}", seed + 1, N_RANDOM_SCHEMA); // Create a random schema with random properties let (test_schema, eq_properties) = create_random_schema(seed as u64)?; // Generate a data that satisfies properties given @@ -4842,13 +5102,4 @@ mod tests { assert!(is_expr_constant_util(&constants, &expr)); Ok(()) } - - #[test] - fn test_generator() -> Result<()> { - let mut gen = Generator::new(vec![2, 2, 2]); - while let Some(elem) = gen.next() { - println!("elem:{:?}", elem); - } - Ok(()) - } } From f38a9c4c3b098660ba1e6294079339b478baddcc Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 23 Nov 2023 13:46:42 +0300 Subject: [PATCH 38/44] Remove leftover code --- datafusion/physical-expr/src/equivalence.rs | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 5b532fe58089..24d32331189c 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1891,18 +1891,6 @@ fn update_ordering( Transformed::Yes(node) } -/// This function constructs a duplicate-free `Vec` by filtering out -/// duplicate entries that have same `PhysicalSortExpr` inside. -pub fn deduplicate_sort_exprs(input: Vec) -> Vec { - let mut output = Vec::::new(); - for item in input { - if !output.iter().any(|req| req.eq(&item)) { - output.push(item); - } - } - output -} - #[cfg(test)] mod tests { use std::ops::Not; From 5e86fe620a58f57fc5e6936677f9ade004c26777 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 23 Nov 2023 14:56:10 +0300 Subject: [PATCH 39/44] Add new test --- datafusion/physical-expr/src/equivalence.rs | 183 +++++++++++++------- 1 file changed, 123 insertions(+), 60 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 24d32331189c..403666f52046 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -16,7 +16,7 @@ // under the License. use std::collections::{HashMap, HashSet}; -use std::hash::Hash; +use std::hash::{Hash, Hasher}; use std::sync::Arc; use crate::expressions::{Column, Literal}; @@ -1287,50 +1287,51 @@ impl EquivalenceProperties { let mut orderings = vec![]; for (source, target) in mapping.iter() { - let relevant_deps = referred_dependencies(&dependency_map, source); - let ordering = get_expr_ordering(source, &relevant_deps); - let options = if let SortProperties::Ordered(options) = ordering { - options - } else { - // expression is not ordered. Try next projection - continue; - }; - let sort_expr = PhysicalSortExpr { - expr: target.clone(), - options, - }; - - // Construct all of the valid prefix orderings for each of the expressions, that is referred in the projection expression. - let relevant_prefixes = relevant_deps - .iter() - .flat_map(|dep| { - let alternative_prefixes = - construct_prefix_orderings(dep, &dependency_map); - (!alternative_prefixes.is_empty()).then_some(alternative_prefixes) - }) - .collect::>(); + for relevant_deps in referred_dependencies(&dependency_map, source) { + let ordering = get_expr_ordering(source, &relevant_deps); + let options = if let SortProperties::Ordered(options) = ordering { + options + } else { + // expression is not ordered. Try next projection + continue; + }; + let sort_expr = PhysicalSortExpr { + expr: target.clone(), + options, + }; - // No dependency, it is a leading ordering - if relevant_prefixes.is_empty() { - orderings.push(vec![sort_expr.clone()]); - } + // Construct all of the valid prefix orderings for each of the expressions, that is referred in the projection expression. + let relevant_prefixes = relevant_deps + .iter() + .flat_map(|dep| { + let alternative_prefixes = + construct_prefix_orderings(dep, &dependency_map); + (!alternative_prefixes.is_empty()).then_some(alternative_prefixes) + }) + .collect::>(); - // Generate all possible orderings where dependencies are satisfied for the current projection expression. - // If expression is a+b ASC, and dependency for a ASC is [c ASC], dependency for b ASC is [d DESC]. - // Then we generate [c ASC, d DESC, a+b ASC], [d DESC, c ASC, a+b ASC]. - for prefix_orderings in - relevant_prefixes.into_iter().multi_cartesian_product() - { - for prefixes in - prefix_orderings.iter().permutations(prefix_orderings.len()) + // No dependency, it is a leading ordering + if relevant_prefixes.is_empty() { + orderings.push(vec![sort_expr.clone()]); + } + + // Generate all possible orderings where dependencies are satisfied for the current projection expression. + // If expression is a+b ASC, and dependency for a ASC is [c ASC], dependency for b ASC is [d DESC]. + // Then we generate [c ASC, d DESC, a+b ASC], [d DESC, c ASC, a+b ASC]. + for prefix_orderings in + relevant_prefixes.into_iter().multi_cartesian_product() { - let ordering = prefixes - .into_iter() - .flatten() - .chain(std::iter::once(&sort_expr)) - .cloned() - .collect(); - orderings.push(ordering); + for prefixes in + prefix_orderings.iter().permutations(prefix_orderings.len()) + { + let ordering = prefixes + .into_iter() + .flatten() + .chain(std::iter::once(&sort_expr)) + .cloned() + .collect(); + orderings.push(ordering); + } } } } @@ -1572,7 +1573,27 @@ fn expr_refers( } } -/// Collests referred dependencies for a given source expression. +/// Wrapper struct for `Arc` +/// this struct can be used as key in the hash map +/// (`Arc` cannot) +#[derive(Debug, Clone)] +struct ExprWrapper(Arc); + +impl PartialEq for ExprWrapper { + fn eq(&self, other: &Self) -> bool { + self.0.eq(&other.0) + } +} + +impl Eq for ExprWrapper {} + +impl Hash for ExprWrapper { + fn hash(&self, state: &mut H) { + self.0.hash(state); + } +} + +/// Collects referred dependencies for a given source expression. /// /// This function analyzes the dependency map to collect referred dependencies for /// a given source expression. @@ -1586,21 +1607,31 @@ fn expr_refers( /// /// # Returns /// -/// A `HashSet` containing relevant dependencies for the given source -/// expression. These dependencies are expressions that are referred to by the source +/// A `Vec>` containing dependencies for the given source +/// expression (Each `HashSet` inside vector can construct source expression). +/// These dependencies are expressions that are referred to by the source /// expression based on the provided dependency map. fn referred_dependencies( dependency_map: &HashMap, source: &Arc, -) -> HashSet { - dependency_map - .iter() - .filter_map(|(sort_expr, _node)| { - if expr_refers(source, &sort_expr.expr) { - Some(sort_expr.clone()) - } else { - None - } +) -> Vec> { + let mut expr_to_sort_exprs: HashMap> = + HashMap::new(); + for sort_expr in dependency_map.keys() { + if expr_refers(source, &sort_expr.expr) { + let key = ExprWrapper(sort_expr.expr.clone()); + let res = expr_to_sort_exprs.entry(key).or_default(); + res.insert(sort_expr.clone()); + } + } + expr_to_sort_exprs + .values() + .multi_cartesian_product() + .map(|referred_deps| { + referred_deps + .into_iter() + .cloned() + .collect::>() }) .collect() } @@ -4179,6 +4210,10 @@ mod tests { descending: false, nulls_first: false, }; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; let test_cases = vec![ // ---------- TEST CASE 1 ------------ @@ -4313,7 +4348,7 @@ mod tests { (col_d, "d_new".to_string()), (&b_plus_d, "b+d".to_string()), ], - // expected, This expected may be missing + // expected vec![ // [a_new ASC, b_new ASC] vec![("a_new", option_asc), ("b_new", option_asc)], @@ -4425,7 +4460,7 @@ mod tests { (col_a, "a_new".to_string()), (&b_plus_d, "b+d".to_string()), ], - // expected, This expected may be missing + // expected vec![ // [a_new ASC, b_new ASC] vec![("a_new", option_asc), ("b_new", option_asc)], @@ -4446,7 +4481,7 @@ mod tests { ], // proj exprs vec![(col_c, "c_new".to_string()), (col_a, "a_new".to_string())], - // expected, This expected may be missing + // expected vec![ // [a_new ASC] vec![("a_new", option_asc)], @@ -4476,7 +4511,7 @@ mod tests { (col_a, "a_new".to_string()), (&a_plus_b, "a+b".to_string()), ], - // expected, This expected may be missing + // expected vec![ // [a_new ASC, b_new ASC, c_new ASC] vec![ @@ -4510,7 +4545,7 @@ mod tests { (col_a, "a_new".to_string()), (&b_plus_e, "b+e".to_string()), ], - // expected, This expected may be missing + // expected vec![ // [a_new ASC, d_new ASC, b+e ASC] vec![ @@ -4555,7 +4590,7 @@ mod tests { (col_a, "a_new".to_string()), (&a_plus_b, "a+b".to_string()), ], - // expected, This expected may be missing + // expected vec![ // [a_new ASC, d_new ASC, b+e ASC] vec![ @@ -4565,6 +4600,34 @@ mod tests { ], ], ), + // ------- TEST CASE 16 ---------- + ( + // orderings + vec![ + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + // [c ASC, b DESC] + vec![(col_c, option_asc), (col_b, option_desc)], + // [e ASC] + vec![(col_e, option_asc)], + ], + // proj exprs + vec![ + (col_c, "c_new".to_string()), + (col_a, "a_new".to_string()), + (col_b, "b_new".to_string()), + (&b_plus_e, "b+e".to_string()), + ], + // expected + vec![ + // [a_new ASC, b_new ASC] + vec![("a_new", option_asc), ("b_new", option_asc)], + // [a_new ASC, b_new ASC] + vec![("a_new", option_asc), ("b+e", option_asc)], + // [c_new ASC, b_new DESC] + vec![("c_new", option_asc), ("b_new", option_desc)], + ], + ), ]; for (idx, (orderings, proj_exprs, expected)) in test_cases.into_iter().enumerate() From 66856ae709160200ecb763875cc4ce00aaf50e27 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 23 Nov 2023 15:22:00 +0300 Subject: [PATCH 40/44] Minor changes --- datafusion/physical-expr/src/equivalence.rs | 23 +++++++++++++++------ 1 file changed, 17 insertions(+), 6 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 403666f52046..a9d0c350a071 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1615,15 +1615,26 @@ fn referred_dependencies( dependency_map: &HashMap, source: &Arc, ) -> Vec> { + // Associate `PhysicalExpr` and `PhysicalSortExpr`s that contain it. + // such as a-> (a ASC, a DESC) let mut expr_to_sort_exprs: HashMap> = HashMap::new(); - for sort_expr in dependency_map.keys() { - if expr_refers(source, &sort_expr.expr) { + dependency_map + .keys() + .filter(|sort_expr| expr_refers(source, &sort_expr.expr)) + .for_each(|sort_expr| { let key = ExprWrapper(sort_expr.expr.clone()); - let res = expr_to_sort_exprs.entry(key).or_default(); - res.insert(sort_expr.clone()); - } - } + expr_to_sort_exprs + .entry(key) + .or_default() + .insert(sort_expr.clone()); + }); + + // Generate all valid dependencies for the source + // such as for a+b + // If map is a-> (a ASC, a DESC). + // b-> (b ASC) + // Generate vec![HashSet(a ASC, b ASC), HashSet(a DESC, b ASC)] expr_to_sort_exprs .values() .multi_cartesian_product() From c2a617ed2c4ec490753a0f4a288a3d9239ae6692 Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Fri, 24 Nov 2023 14:18:43 +0300 Subject: [PATCH 41/44] Review Part 1 --- datafusion/physical-expr/src/equivalence.rs | 25 +++++++++------------ 1 file changed, 11 insertions(+), 14 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index a9d0c350a071..bab75b373275 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1202,21 +1202,18 @@ impl EquivalenceProperties { let mut dependency_map = HashMap::new(); for ordering in self.normalized_oeq_class().iter() { for (idx, sort_expr) in ordering.iter().enumerate() { - // Previous ordering is dependency. For leading ordering (e.g first sort_expr inside ordering) - // there is no dependency. - // let dependency = (idx > 0).then_some(&ordering[idx - 1]); - let dependency = if idx > 0 { - Some(&ordering[idx - 1]) - } else { - None - }; - let target_expr = self.project_expr(&sort_expr.expr, mapping); - let target_sort_expr = target_expr.map(|expr| PhysicalSortExpr { - expr, - options: sort_expr.options, - }); + let target_sort_expr = + self.project_expr(&sort_expr.expr, mapping).map(|expr| { + PhysicalSortExpr { + expr, + options: sort_expr.options, + } + }); let is_projected = target_sort_expr.is_some(); - if is_projected | any_projection_refers(mapping, &sort_expr.expr) { + if is_projected || any_projection_refers(mapping, &sort_expr.expr) { + // Previous ordering is dependency. For leading ordering (e.g first sort_expr inside ordering) + // there is no dependency. + let dependency = idx.checked_sub(1).map(|a| &ordering[a]); // Add sort_exprs that // - can be projected // - referred by any of the projection expressions From 68586e81a888669fa288bf967d13ab2bd08f7021 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 24 Nov 2023 15:47:20 +0300 Subject: [PATCH 42/44] simplifications --- datafusion/physical-expr/src/equivalence.rs | 248 ++++++++++++-------- 1 file changed, 146 insertions(+), 102 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index bab75b373275..e80aaf12f9d8 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1187,17 +1187,17 @@ impl EquivalenceProperties { /// /// # Returns /// - /// A `HashMap` representing the dependency map, where each + /// A [`DependencyMap`] representing the dependency map, where each /// [`DependencyNode`] contains dependencies for the key [`PhysicalSortExpr`]. /// - /// As an example: Assume mapping is [a->a_new, b->b_new, c->c_new], Ordering [a ASC, b ASC, c DESC] will be transformed to the - /// a ASC: Node{a_new ASC, HashSet{}} - /// b ASC: Node{b_new ASC, HashSet{a ASC}} - /// c DESC: Node{c_new DESC, HashSet{b ASC}}. - fn construct_dependency_map( - &self, - mapping: &ProjectionMapping, - ) -> HashMap { + /// As an example: Assume: + /// Orderings are: [a ASC, b ASC], [a ASC, c ASC] + /// mapping is [a->a_new, b->b_new, b+c -> b+c], + /// Dependency map will be + /// a ASC: Node{Some(a_new ASC), HashSet{}} + /// b ASC: Node{Some(b_new ASC), HashSet{a ASC}} + /// c ASC: Node{None, HashSet{a ASC}}. + fn construct_dependency_map(&self, mapping: &ProjectionMapping) -> DependencyMap { // Construct dependency map of the existing orderings that are referred in the projection. let mut dependency_map = HashMap::new(); for ordering in self.normalized_oeq_class().iter() { @@ -1224,7 +1224,7 @@ impl EquivalenceProperties { target_sort_expr: target_sort_expr.clone(), dependencies: HashSet::new(), }) - .insert(dependency); + .insert_dependency(dependency); } if !is_projected { // If expression cannot be projected @@ -1282,76 +1282,68 @@ impl EquivalenceProperties { // Get dependency map for existing orderings. let dependency_map = self.construct_dependency_map(&mapping); - let mut orderings = vec![]; - for (source, target) in mapping.iter() { - for relevant_deps in referred_dependencies(&dependency_map, source) { - let ordering = get_expr_ordering(source, &relevant_deps); - let options = if let SortProperties::Ordered(options) = ordering { - options - } else { - // expression is not ordered. Try next projection - continue; - }; - let sort_expr = PhysicalSortExpr { - expr: target.clone(), - options, - }; - - // Construct all of the valid prefix orderings for each of the expressions, that is referred in the projection expression. - let relevant_prefixes = relevant_deps - .iter() - .flat_map(|dep| { - let alternative_prefixes = - construct_prefix_orderings(dep, &dependency_map); - (!alternative_prefixes.is_empty()).then_some(alternative_prefixes) + let mut orderings = mapping + .iter() + .flat_map(|(source, target)| { + referred_dependencies(&dependency_map, source) + .into_iter() + .filter_map(|relevant_deps| { + if let SortProperties::Ordered(options) = + get_expr_ordering(source, &relevant_deps) + { + Some((options, relevant_deps)) + } else { + // do not consider unordered cases + None + } }) - .collect::>(); - - // No dependency, it is a leading ordering - if relevant_prefixes.is_empty() { - orderings.push(vec![sort_expr.clone()]); - } - - // Generate all possible orderings where dependencies are satisfied for the current projection expression. - // If expression is a+b ASC, and dependency for a ASC is [c ASC], dependency for b ASC is [d DESC]. - // Then we generate [c ASC, d DESC, a+b ASC], [d DESC, c ASC, a+b ASC]. - for prefix_orderings in - relevant_prefixes.into_iter().multi_cartesian_product() - { - for prefixes in - prefix_orderings.iter().permutations(prefix_orderings.len()) - { - let ordering = prefixes + .flat_map(|(options, relevant_deps)| { + // Ordered expression, and its dependencies + let sort_expr = PhysicalSortExpr { + expr: target.clone(), + options, + }; + + // Generate dependent orderings (e.g prefixes for the `sort_expr`). + let dependency_orderings = generate_dependency_orderings( + &relevant_deps, + &dependency_map, + ); + // Suffix sort_expr to the end of the dependent orderings + dependency_orderings .into_iter() - .flatten() - .chain(std::iter::once(&sort_expr)) - .cloned() - .collect(); - orderings.push(ordering); - } - } - } - } + .map(|mut ordering| { + ordering.push(sort_expr.clone()); + ordering + }) + .collect::>() + }) + }) + .collect::>(); - // Add valid projected orderings. - for (sort_expr, node) in dependency_map.iter() { - let mut prefixes = construct_prefix_orderings(sort_expr, &dependency_map); - if prefixes.is_empty() { - // If prefix is empty, there is no dependency. - // `node.target_sort_expr` is itself leading ordering, add it to the orderings. - if let Some(target) = &node.target_sort_expr { - orderings.push(vec![target.clone()]); - } - } - // Append current ordering on top its dependencies - prefixes.iter_mut().for_each(|ordering| { - if let Some(target) = &node.target_sort_expr { - ordering.push(target.clone()) + // Add valid projected orderings. Such as if existing ordering is a+b and projection is (a-> a_new, b->b_new) + // We need to preserve (a_new+b_new) as ordered. Please note that a_new and b_new themselves are not ordered. + // This dependency cannot be understood from the calculations above. + let projected_orderings = dependency_map + .iter() + .flat_map(|(sort_expr, node)| { + let mut prefixes = construct_prefix_orderings(sort_expr, &dependency_map); + if prefixes.is_empty() { + // If prefix is empty, there is no dependency. Insert empty ordering + prefixes = vec![vec![]]; } - }); - orderings.extend(prefixes); - } + // Append current ordering on top its dependencies + prefixes.iter_mut().for_each(|ordering| { + if let Some(target) = &node.target_sort_expr { + ordering.push(target.clone()) + } + }); + prefixes + }) + .collect::>(); + orderings.extend(projected_orderings); + // Simplify each ordering by removing redundant sections. orderings.into_iter().map(collapse_lex_ordering).collect() } @@ -1560,14 +1552,11 @@ fn expr_refers( referring_expr: &Arc, referred_expr: &Arc, ) -> bool { - if referring_expr.eq(referred_expr) { - true - } else { - referring_expr + referring_expr.eq(referred_expr) + || referring_expr .children() .iter() .any(|child| expr_refers(child, referred_expr)) - } } /// Wrapper struct for `Arc` @@ -1597,25 +1586,24 @@ impl Hash for ExprWrapper { /// /// # Parameters /// -/// - `dependency_map`: A reference to the `HashMap` representing -/// the dependency map, where each expression is associated with a `Node`. +/// - `dependency_map`: A reference to the `DependencyMap` representing +/// the dependency map, where each `PhysicalSortExpr` is associated with a `DependencyMap`. /// - `source`: A reference to the source expression (`Arc`) for which /// relevant dependencies need to be identified. /// /// # Returns /// -/// A `Vec>` containing dependencies for the given source -/// expression (Each `HashSet` inside vector can construct source expression). +/// A `Vec` containing dependencies for the given source +/// expression (Each `Dependencies` inside the vector is sufficient (and not redundant) to construct source expression). /// These dependencies are expressions that are referred to by the source /// expression based on the provided dependency map. fn referred_dependencies( - dependency_map: &HashMap, + dependency_map: &DependencyMap, source: &Arc, -) -> Vec> { +) -> Vec { // Associate `PhysicalExpr` and `PhysicalSortExpr`s that contain it. // such as a-> (a ASC, a DESC) - let mut expr_to_sort_exprs: HashMap> = - HashMap::new(); + let mut expr_to_sort_exprs: HashMap = HashMap::new(); dependency_map .keys() .filter(|sort_expr| expr_refers(source, &sort_expr.expr)) @@ -1654,7 +1642,7 @@ fn referred_dependencies( /// /// - `referred_sort_expr`: A reference to the relevant sort expression (`PhysicalSortExpr`) /// for which lexicographical orderings need to be constructed, that satisfying its dependencies. -/// - `dependency_map`: A reference to the `HashMap` that contains dependencies +/// - `dependency_map`: A reference to the `DependencyMap` that contains dependencies /// for different `PhysicalSortExpr`s. /// /// # Returns @@ -1663,7 +1651,7 @@ fn referred_dependencies( /// sort expression and its dependencies. fn construct_orderings( referred_sort_expr: &PhysicalSortExpr, - dependency_map: &HashMap, + dependency_map: &DependencyMap, ) -> Vec { // We are sure that `referred_sort_expr` is inside `dependency_map`. let val = &dependency_map[referred_sort_expr]; @@ -1695,7 +1683,7 @@ fn construct_orderings( /// /// - `relevant_sort_expr`: A reference to the relevant sort expression (`PhysicalSortExpr`) /// for which prefix orderings need to be constructed. -/// - `dependency_map`: A reference to the `HashMap` that contains dependencies +/// - `dependency_map`: A reference to the `DependencyMap` that contains dependencies /// for different `PhysicalSortExpr`s. /// /// # Returns @@ -1704,7 +1692,7 @@ fn construct_orderings( /// expression and its dependencies. fn construct_prefix_orderings( relevant_sort_expr: &PhysicalSortExpr, - dependency_map: &HashMap, + dependency_map: &DependencyMap, ) -> Vec { dependency_map[relevant_sort_expr] .dependencies @@ -1713,6 +1701,62 @@ fn construct_prefix_orderings( .collect() } +/// Generates Dependency Orderings (orderings that satisfy dependencies) +/// +/// Given a set of relevant dependencies (`relevant_deps`) and a map of dependencies +/// (`dependency_map`), this function generates all possible prefix orderings based on +/// the dependencies. +/// +/// # Arguments +/// +/// * `dependencies` - A reference to the dependencies. +/// * `dependency_map` - A reference to the map of dependencies for expressions. +/// +/// # Returns +/// +/// A vector of lexical orderings (`Vec`) representing all valid orderings +/// based on the given dependencies. +fn generate_dependency_orderings( + dependencies: &Dependencies, + dependency_map: &DependencyMap, +) -> Vec { + // Construct all of the valid prefix orderings for each of the expressions, that is referred in the projection expression. + let relevant_prefixes = dependencies + .iter() + .flat_map(|dep| { + let prefixes = construct_prefix_orderings(dep, dependency_map); + (!prefixes.is_empty()).then_some(prefixes) + }) + .collect::>(); + + // No dependency, dependent is leading ordering. + if relevant_prefixes.is_empty() { + // Return an empty ordering. + return vec![vec![]]; + } + + // Generate all possible orderings where dependencies are satisfied for the current projection expression. + // If expression is a+b ASC, and dependency for a ASC is [c ASC], dependency for b ASC is [d DESC]. + // Then we generate [c ASC, d DESC, a+b ASC], [d DESC, c ASC, a+b ASC]. + relevant_prefixes + .into_iter() + .multi_cartesian_product() + .flat_map(|prefix_orderings| { + prefix_orderings + .iter() + .permutations(prefix_orderings.len()) + .map(|prefixes| { + prefixes + .into_iter() + .flatten() + .cloned() + .collect::() + }) + .collect::>() + }) + .collect::>() +} + /// Retrieves the ordering properties for a given expression based on sort expressions it refers. /// /// This function examines the provided expression and sort expressions it refers to determine @@ -1722,7 +1766,7 @@ fn construct_prefix_orderings( /// /// - `expr`: A reference to the source expression (`Arc`) for which ordering /// properties need to be determined. -/// - `referred_sort_exprs`: A reference to a `HashSet` containing sort +/// - `dependencies`: A reference to a `Dependencies` containing sort /// expressions referred by the `expr`. /// /// # Returns @@ -1730,12 +1774,9 @@ fn construct_prefix_orderings( /// A `SortProperties` enum indicating the ordering information of the given expression: fn get_expr_ordering( expr: &Arc, - referred_sort_exprs: &HashSet, + dependencies: &Dependencies, ) -> SortProperties { - if let Some(column_order) = referred_sort_exprs - .iter() - .find(|&order| expr.eq(&order.expr)) - { + if let Some(column_order) = dependencies.iter().find(|&order| expr.eq(&order.expr)) { // If exact match is found, return its ordering. SortProperties::Ordered(column_order.options) } else { @@ -1743,7 +1784,7 @@ fn get_expr_ordering( let child_states = expr .children() .iter() - .map(|child| get_expr_ordering(child, referred_sort_exprs)) + .map(|child| get_expr_ordering(child, dependencies)) .collect::>(); // Calculate expression ordering using ordering of its children. expr.get_ordering(&child_states) @@ -1780,23 +1821,26 @@ fn any_projection_refers( /// /// - `target_sort_expr`: An optional `PhysicalSortExpr` representing the target sort expression /// associated with the node. It is `None` if the sort expression cannot be projected. -/// - `dependencies`: A `HashSet` containing dependencies on other sort expressions +/// - `dependencies`: A [`Dependencies`] containing dependencies on other sort expressions /// that are referred to by the target sort expression. #[derive(Debug, Clone, PartialEq, Eq)] struct DependencyNode { target_sort_expr: Option, - dependencies: HashSet, + dependencies: Dependencies, } impl DependencyNode { // Insert dependency to the state, (if there is any dependency) - fn insert(&mut self, dependency: Option<&PhysicalSortExpr>) { + fn insert_dependency(&mut self, dependency: Option<&PhysicalSortExpr>) { if let Some(dep) = dependency { self.dependencies.insert(dep.clone()); } } } +type DependencyMap = HashMap; +type Dependencies = HashSet; + /// Calculate ordering equivalence properties for the given join operation. pub fn join_equivalence_properties( left: EquivalenceProperties, From 15089648f2ce012fa157c2fecc96f19fa0ada64a Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 8 Dec 2023 18:06:38 +0300 Subject: [PATCH 43/44] Review --- datafusion/physical-expr/src/equivalence.rs | 418 +++++++++----------- 1 file changed, 188 insertions(+), 230 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index e80aaf12f9d8..a02eeeeddc0d 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -27,6 +27,7 @@ use crate::{ LexRequirement, LexRequirementRef, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; + use arrow::datatypes::SchemaRef; use arrow_schema::SortOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; @@ -1173,32 +1174,36 @@ impl EquivalenceProperties { self.eq_group.project_expr(projection_mapping, expr) } - /// Constructs a dependency map based on existing orderings referred to in the projection. + /// Constructs a dependency map based on existing orderings referred to in + /// the projection. /// - /// This function analyzes the orderings in the normalized order-equivalence class and - /// builds a dependency map. The dependency map captures relationships between expressions - /// within the orderings, helping to identify dependencies and construct valid projected - /// orderings during projection operations. + /// This function analyzes the orderings in the normalized order-equivalence + /// class and builds a dependency map. The dependency map captures relationships + /// between expressions within the orderings, helping to identify dependencies + /// and construct valid projected orderings during projection operations. /// /// # Parameters /// - /// - `mapping`: A reference to the `ProjectionMapping` that defines the relationship - /// between source and target expressions. + /// - `mapping`: A reference to the `ProjectionMapping` that defines the + /// relationship between source and target expressions. /// /// # Returns /// /// A [`DependencyMap`] representing the dependency map, where each /// [`DependencyNode`] contains dependencies for the key [`PhysicalSortExpr`]. /// - /// As an example: Assume: - /// Orderings are: [a ASC, b ASC], [a ASC, c ASC] - /// mapping is [a->a_new, b->b_new, b+c -> b+c], - /// Dependency map will be - /// a ASC: Node{Some(a_new ASC), HashSet{}} - /// b ASC: Node{Some(b_new ASC), HashSet{a ASC}} - /// c ASC: Node{None, HashSet{a ASC}}. + /// # Example + /// + /// Assume we have two equivalent orderings: `[a ASC, b ASC]` and `[a ASC, c ASC]`, + /// and the projection mapping is `[a -> a_new, b -> b_new, b + c -> b + c]`. + /// Then, the dependency map will be: + /// + /// ```text + /// a ASC: Node {Some(a_new ASC), HashSet{}} + /// b ASC: Node {Some(b_new ASC), HashSet{a ASC}} + /// c ASC: Node {None, HashSet{a ASC}} + /// ``` fn construct_dependency_map(&self, mapping: &ProjectionMapping) -> DependencyMap { - // Construct dependency map of the existing orderings that are referred in the projection. let mut dependency_map = HashMap::new(); for ordering in self.normalized_oeq_class().iter() { for (idx, sort_expr) in ordering.iter().enumerate() { @@ -1210,14 +1215,17 @@ impl EquivalenceProperties { } }); let is_projected = target_sort_expr.is_some(); - if is_projected || any_projection_refers(mapping, &sort_expr.expr) { - // Previous ordering is dependency. For leading ordering (e.g first sort_expr inside ordering) - // there is no dependency. + if is_projected + || mapping + .iter() + .any(|(source, _)| expr_refers(source, &sort_expr.expr)) + { + // Previous ordering is a dependency. Note that there is no, + // dependency for a leading ordering (i.e. the first sort + // expression). let dependency = idx.checked_sub(1).map(|a| &ordering[a]); - // Add sort_exprs that - // - can be projected - // - referred by any of the projection expressions - // to the dependency map. + // Add sort expressions that can be projected or referred to + // by any of the projection expressions to the dependency map: dependency_map .entry(sort_expr.clone()) .or_insert_with(|| DependencyNode { @@ -1227,8 +1235,8 @@ impl EquivalenceProperties { .insert_dependency(dependency); } if !is_projected { - // If expression cannot be projected - // Stop constructing dependency. Remaining dependencies will not be valid after projection. + // If we can not project, stop constructing the dependency + // map as remaining dependencies will be invalid after projection. break; } } @@ -1238,9 +1246,9 @@ impl EquivalenceProperties { /// Returns a new `ProjectionMapping` where source expressions are normalized. /// - /// The normalization ensures that source expressions are transformed into a consistent - /// representation. This is beneficial for algorithms that rely on exact equalities, - /// as it allows for more precise and reliable comparisons. + /// This normalization ensures that source expressions are transformed into a + /// consistent representation. This is beneficial for algorithms that rely on + /// exact equalities, as it allows for more precise and reliable comparisons. /// /// # Parameters /// @@ -1252,99 +1260,93 @@ impl EquivalenceProperties { fn normalized_mapping(&self, mapping: &ProjectionMapping) -> ProjectionMapping { // Construct the mapping where source expressions are normalized. In this way // In the algorithms below we can work on exact equalities - let new_inner = mapping - .iter() - .map(|(source, target)| { - let normalized_source = self.eq_group.normalize_expr(source.clone()); - (normalized_source, target.clone()) - }) - .collect::>(); - ProjectionMapping { map: new_inner } + ProjectionMapping { + map: mapping + .iter() + .map(|(source, target)| { + let normalized_source = self.eq_group.normalize_expr(source.clone()); + (normalized_source, target.clone()) + }) + .collect(), + } } /// Computes projected orderings based on a given projection mapping. /// - /// This function takes a `ProjectionMapping` and computes the possible orderings - /// for the projected expressions. It considers dependencies between expressions - /// and generates valid orderings according to the specified sort properties. + /// This function takes a `ProjectionMapping` and computes the possible + /// orderings for the projected expressions. It considers dependencies + /// between expressions and generates valid orderings according to the + /// specified sort properties. /// /// # Parameters /// - /// - `mapping`: A reference to the `ProjectionMapping` that defines the relationship - /// between source and target expressions. + /// - `mapping`: A reference to the `ProjectionMapping` that defines the + /// relationship between source and target expressions. /// /// # Returns /// - /// A vector of `LexOrdering` containing all of the valid orderings after projection. + /// A vector of `LexOrdering` containing all valid orderings after projection. fn projected_orderings(&self, mapping: &ProjectionMapping) -> Vec { let mapping = self.normalized_mapping(mapping); - // Get dependency map for existing orderings. + // Get dependency map for existing orderings: let dependency_map = self.construct_dependency_map(&mapping); - let mut orderings = mapping - .iter() - .flat_map(|(source, target)| { - referred_dependencies(&dependency_map, source) - .into_iter() - .filter_map(|relevant_deps| { - if let SortProperties::Ordered(options) = - get_expr_ordering(source, &relevant_deps) - { - Some((options, relevant_deps)) - } else { - // do not consider unordered cases - None - } - }) - .flat_map(|(options, relevant_deps)| { - // Ordered expression, and its dependencies - let sort_expr = PhysicalSortExpr { - expr: target.clone(), - options, - }; - - // Generate dependent orderings (e.g prefixes for the `sort_expr`). - let dependency_orderings = generate_dependency_orderings( - &relevant_deps, - &dependency_map, - ); - // Suffix sort_expr to the end of the dependent orderings - dependency_orderings - .into_iter() - .map(|mut ordering| { - ordering.push(sort_expr.clone()); - ordering - }) - .collect::>() - }) - }) - .collect::>(); + let orderings = mapping.iter().flat_map(|(source, target)| { + referred_dependencies(&dependency_map, source) + .into_iter() + .filter_map(|relevant_deps| { + if let SortProperties::Ordered(options) = + get_expr_ordering(source, &relevant_deps) + { + Some((options, relevant_deps)) + } else { + // Do not consider unordered cases + None + } + }) + .flat_map(|(options, relevant_deps)| { + let sort_expr = PhysicalSortExpr { + expr: target.clone(), + options, + }; + // Generate dependent orderings (i.e. prefixes for `sort_expr`): + let mut dependency_orderings = + generate_dependency_orderings(&relevant_deps, &dependency_map); + // Append `sort_expr` to the dependent orderings: + for ordering in dependency_orderings.iter_mut() { + ordering.push(sort_expr.clone()); + } + dependency_orderings + }) + }); - // Add valid projected orderings. Such as if existing ordering is a+b and projection is (a-> a_new, b->b_new) - // We need to preserve (a_new+b_new) as ordered. Please note that a_new and b_new themselves are not ordered. - // This dependency cannot be understood from the calculations above. - let projected_orderings = dependency_map - .iter() - .flat_map(|(sort_expr, node)| { - let mut prefixes = construct_prefix_orderings(sort_expr, &dependency_map); - if prefixes.is_empty() { - // If prefix is empty, there is no dependency. Insert empty ordering - prefixes = vec![vec![]]; + // Add valid projected orderings. For example, if existing ordering is + // `a + b` and projection is `[a -> a_new, b -> b_new]`, we need to + // preserve `a_new + b_new` as ordered. Please note that `a_new` and + // `b_new` themselves need not be ordered. Such dependencies cannot be + // deduced via the pass above. + let projected_orderings = dependency_map.iter().flat_map(|(sort_expr, node)| { + let mut prefixes = construct_prefix_orderings(sort_expr, &dependency_map); + if prefixes.is_empty() { + // If prefix is empty, there is no dependency. Insert + // empty ordering: + prefixes = vec![vec![]]; + } + // Append current ordering on top its dependencies: + for ordering in prefixes.iter_mut() { + if let Some(target) = &node.target_sort_expr { + ordering.push(target.clone()) } - // Append current ordering on top its dependencies - prefixes.iter_mut().for_each(|ordering| { - if let Some(target) = &node.target_sort_expr { - ordering.push(target.clone()) - } - }); - prefixes - }) - .collect::>(); + } + prefixes + }); - orderings.extend(projected_orderings); - // Simplify each ordering by removing redundant sections. - orderings.into_iter().map(collapse_lex_ordering).collect() + // Simplify each ordering by removing redundant sections: + orderings + .chain(projected_orderings) + .map(collapse_lex_ordering) + .collect() } /// Projects constants based on the provided `ProjectionMapping`. @@ -1533,8 +1535,6 @@ fn is_constant_recurse( !children.is_empty() && children.iter().all(|c| is_constant_recurse(constants, c)) } -/// Checks if a referring expression refers to a given referred expression. -/// /// This function examines whether a referring expression directly refers to a /// given referred expression or if any of its children in the expression tree /// refer to the specified expression. @@ -1559,9 +1559,7 @@ fn expr_refers( .any(|child| expr_refers(child, referred_expr)) } -/// Wrapper struct for `Arc` -/// this struct can be used as key in the hash map -/// (`Arc` cannot) +/// Wrapper struct for `Arc` to use them as keys in a hash map. #[derive(Debug, Clone)] struct ExprWrapper(Arc); @@ -1579,117 +1577,103 @@ impl Hash for ExprWrapper { } } -/// Collects referred dependencies for a given source expression. -/// /// This function analyzes the dependency map to collect referred dependencies for /// a given source expression. /// /// # Parameters /// -/// - `dependency_map`: A reference to the `DependencyMap` representing -/// the dependency map, where each `PhysicalSortExpr` is associated with a `DependencyMap`. -/// - `source`: A reference to the source expression (`Arc`) for which -/// relevant dependencies need to be identified. +/// - `dependency_map`: A reference to the `DependencyMap` where each +/// `PhysicalSortExpr` is associated with a `DependencyNode`. +/// - `source`: A reference to the source expression (`Arc`) +/// for which relevant dependencies need to be identified. /// /// # Returns /// -/// A `Vec` containing dependencies for the given source -/// expression (Each `Dependencies` inside the vector is sufficient (and not redundant) to construct source expression). -/// These dependencies are expressions that are referred to by the source -/// expression based on the provided dependency map. +/// A `Vec` containing the dependencies for the given source +/// expression. These dependencies are expressions that are referred to by +/// the source expression based on the provided dependency map. fn referred_dependencies( dependency_map: &DependencyMap, source: &Arc, ) -> Vec { - // Associate `PhysicalExpr` and `PhysicalSortExpr`s that contain it. - // such as a-> (a ASC, a DESC) - let mut expr_to_sort_exprs: HashMap = HashMap::new(); - dependency_map + // Associate `PhysicalExpr`s with `PhysicalSortExpr`s that contain them: + let mut expr_to_sort_exprs = HashMap::::new(); + for sort_expr in dependency_map .keys() .filter(|sort_expr| expr_refers(source, &sort_expr.expr)) - .for_each(|sort_expr| { - let key = ExprWrapper(sort_expr.expr.clone()); - expr_to_sort_exprs - .entry(key) - .or_default() - .insert(sort_expr.clone()); - }); + { + let key = ExprWrapper(sort_expr.expr.clone()); + expr_to_sort_exprs + .entry(key) + .or_default() + .insert(sort_expr.clone()); + } - // Generate all valid dependencies for the source - // such as for a+b - // If map is a-> (a ASC, a DESC). - // b-> (b ASC) - // Generate vec![HashSet(a ASC, b ASC), HashSet(a DESC, b ASC)] + // Generate all valid dependencies for the source. For example, if the source + // is `a + b` and the map is `[a -> (a ASC, a DESC), b -> (b ASC)]`, we get + // `vec![HashSet(a ASC, b ASC), HashSet(a DESC, b ASC)]`. expr_to_sort_exprs .values() .multi_cartesian_product() - .map(|referred_deps| { - referred_deps - .into_iter() - .cloned() - .collect::>() - }) + .map(|referred_deps| referred_deps.into_iter().cloned().collect()) .collect() } -/// Constructs Lexicographical Orderings based on the given relevant sort expression and dependency map. -/// -/// This function recursively analyzes the dependencies of the provided relevant sort expression -/// within the given dependency map. It constructs lexicographical orderings that include the -/// relevant sort expression and its dependencies. +/// This function recursively analyzes the dependencies of the given sort +/// expression within the given dependency map to construct lexicographical +/// orderings that include the sort expression and its dependencies. /// /// # Parameters /// -/// - `referred_sort_expr`: A reference to the relevant sort expression (`PhysicalSortExpr`) -/// for which lexicographical orderings need to be constructed, that satisfying its dependencies. -/// - `dependency_map`: A reference to the `DependencyMap` that contains dependencies -/// for different `PhysicalSortExpr`s. +/// - `referred_sort_expr`: A reference to the sort expression (`PhysicalSortExpr`) +/// for which lexicographical orderings satisfying its dependencies are to be +/// constructed. +/// - `dependency_map`: A reference to the `DependencyMap` that contains +/// dependencies for different `PhysicalSortExpr`s. /// /// # Returns /// -/// A vector of lexicographical orderings (`Vec`) based on the provided relevant +/// A vector of lexicographical orderings (`Vec`) based on the given /// sort expression and its dependencies. fn construct_orderings( referred_sort_expr: &PhysicalSortExpr, dependency_map: &DependencyMap, ) -> Vec { // We are sure that `referred_sort_expr` is inside `dependency_map`. - let val = &dependency_map[referred_sort_expr]; - // Since these nodes are intermediate nodes. We are sure `val.target_sort_expr` is `Some(_)`; - let target_sort_expr = val.target_sort_expr.clone().unwrap(); - if val.dependencies.is_empty() { + let node = &dependency_map[referred_sort_expr]; + // Since we work on intermediate nodes, we are sure `val.target_sort_expr` + // exists. + let target_sort_expr = node.target_sort_expr.clone().unwrap(); + if node.dependencies.is_empty() { vec![vec![target_sort_expr]] } else { - val.dependencies + node.dependencies .iter() .flat_map(|dep| { let mut orderings = construct_orderings(dep, dependency_map); - orderings - .iter_mut() - .for_each(|ordering| ordering.push(target_sort_expr.clone())); + for ordering in orderings.iter_mut() { + ordering.push(target_sort_expr.clone()) + } orderings }) - .collect::>() + .collect() } } -/// Constructs prefix orderings (dependent orderings) based on the given relevant sort expression and dependency map. -/// -/// This function retrieves the dependencies of the provided relevant sort expression from the -/// given dependency map. It then constructs prefix orderings by recursively analyzing the -/// dependencies and including them in the orderings. +/// This function retrieves the dependencies of the given relevant sort expression +/// from the given dependency map. It then constructs prefix orderings by recursively +/// analyzing the dependencies and include them in the orderings. /// /// # Parameters /// -/// - `relevant_sort_expr`: A reference to the relevant sort expression (`PhysicalSortExpr`) -/// for which prefix orderings need to be constructed. -/// - `dependency_map`: A reference to the `DependencyMap` that contains dependencies -/// for different `PhysicalSortExpr`s. +/// - `relevant_sort_expr`: A reference to the relevant sort expression +/// (`PhysicalSortExpr`) for which prefix orderings are to be constructed. +/// - `dependency_map`: A reference to the `DependencyMap` containing dependencies. /// /// # Returns /// -/// A vector of prefix orderings (`Vec`) based on the provided relevant sort -/// expression and its dependencies. +/// A vector of prefix orderings (`Vec`) based on the given relevant +/// sort expression and its dependencies. fn construct_prefix_orderings( relevant_sort_expr: &PhysicalSortExpr, dependency_map: &DependencyMap, @@ -1701,13 +1685,11 @@ fn construct_prefix_orderings( .collect() } -/// Generates Dependency Orderings (orderings that satisfy dependencies) -/// /// Given a set of relevant dependencies (`relevant_deps`) and a map of dependencies -/// (`dependency_map`), this function generates all possible prefix orderings based on -/// the dependencies. +/// (`dependency_map`), this function generates all possible prefix orderings +/// based on the given dependencies. /// -/// # Arguments +/// # Parameters /// /// * `dependencies` - A reference to the dependencies. /// * `dependency_map` - A reference to the map of dependencies for expressions. @@ -1720,7 +1702,8 @@ fn generate_dependency_orderings( dependencies: &Dependencies, dependency_map: &DependencyMap, ) -> Vec { - // Construct all of the valid prefix orderings for each of the expressions, that is referred in the projection expression. + // Construct all the valid prefix orderings for each expression appearing + // in the projection: let relevant_prefixes = dependencies .iter() .flat_map(|dep| { @@ -1729,15 +1712,17 @@ fn generate_dependency_orderings( }) .collect::>(); - // No dependency, dependent is leading ordering. + // No dependency, dependent is a leading ordering. if relevant_prefixes.is_empty() { - // Return an empty ordering. + // Return an empty ordering: return vec![vec![]]; } - // Generate all possible orderings where dependencies are satisfied for the current projection expression. - // If expression is a+b ASC, and dependency for a ASC is [c ASC], dependency for b ASC is [d DESC]. - // Then we generate [c ASC, d DESC, a+b ASC], [d DESC, c ASC, a+b ASC]. + // Generate all possible orderings where dependencies are satisfied for the + // current projection expression. For example, if expression is `a + b ASC`, + // and the dependency for `a ASC` is `[c ASC]`, the dependency for `b ASC` + // is `[d DESC]`, then we generate `[c ASC, d DESC, a + b ASC]` and + // `[d DESC, c ASC, a + b ASC]`. relevant_prefixes .into_iter() .multi_cartesian_product() @@ -1745,33 +1730,25 @@ fn generate_dependency_orderings( prefix_orderings .iter() .permutations(prefix_orderings.len()) - .map(|prefixes| { - prefixes - .into_iter() - .flatten() - .cloned() - .collect::() - }) - .collect::>() + .map(|prefixes| prefixes.into_iter().flatten().cloned().collect()) + .collect::>() }) - .collect::>() + .collect() } -/// Retrieves the ordering properties for a given expression based on sort expressions it refers. -/// -/// This function examines the provided expression and sort expressions it refers to determine -/// the ordering properties of the expression. +/// This function examines the given expression and the sort expressions it +/// refers to determine the ordering properties of the expression. /// /// # Parameters /// -/// - `expr`: A reference to the source expression (`Arc`) for which ordering -/// properties need to be determined. -/// - `dependencies`: A reference to a `Dependencies` containing sort -/// expressions referred by the `expr`. +/// - `expr`: A reference to the source expression (`Arc`) for +/// which ordering properties need to be determined. +/// - `dependencies`: A reference to `Dependencies`, containing sort expressions +/// referred to by `expr`. /// /// # Returns /// -/// A `SortProperties` enum indicating the ordering information of the given expression: +/// A `SortProperties` indicating the ordering information of the given expression. fn get_expr_ordering( expr: &Arc, dependencies: &Dependencies, @@ -1791,38 +1768,19 @@ fn get_expr_ordering( } } -/// Checks if any expression in the projection mapping refers to a given source expression. -/// -/// # Parameters -/// -/// - `mapping`: A reference to the `ProjectionMapping` that defines the relationship between -/// source and target expressions. -/// - `expr`: A reference to the `Arc` for which we check whether any projection -/// source expression refers to it.. -/// -/// # Returns -/// -/// A `bool` flag, that indicates, whether any projection source refers `expr` or not. -fn any_projection_refers( - mapping: &ProjectionMapping, - expr: &Arc, -) -> bool { - mapping - .iter() - .any(|(source, _target)| expr_refers(source, expr)) -} - -/// Represents a node in the dependency map used for constructing projected orderings. +/// Represents a node in the dependency map used to construct projected orderings. /// -/// A `DependencyNode` contains information about a particular sort expression, including -/// its target sort expression and a set of dependencies on other sort expressions. +/// A `DependencyNode` contains information about a particular sort expression, +/// including its target sort expression and a set of dependencies on other sort +/// expressions. /// /// # Fields /// -/// - `target_sort_expr`: An optional `PhysicalSortExpr` representing the target sort expression -/// associated with the node. It is `None` if the sort expression cannot be projected. -/// - `dependencies`: A [`Dependencies`] containing dependencies on other sort expressions -/// that are referred to by the target sort expression. +/// - `target_sort_expr`: An optional `PhysicalSortExpr` representing the target +/// sort expression associated with the node. It is `None` if the sort expression +/// cannot be projected. +/// - `dependencies`: A [`Dependencies`] containing dependencies on other sort +/// expressions that are referred to by the target sort expression. #[derive(Debug, Clone, PartialEq, Eq)] struct DependencyNode { target_sort_expr: Option, @@ -1830,7 +1788,7 @@ struct DependencyNode { } impl DependencyNode { - // Insert dependency to the state, (if there is any dependency) + // Insert dependency to the state (if exists). fn insert_dependency(&mut self, dependency: Option<&PhysicalSortExpr>) { if let Some(dep) = dependency { self.dependencies.insert(dep.clone()); @@ -2154,7 +2112,7 @@ mod tests { .map(|(expr, options)| { PhysicalSortRequirement::new((*expr).clone(), *options) }) - .collect::>() + .collect() } // Convert each tuple to PhysicalSortExpr @@ -2167,7 +2125,7 @@ mod tests { expr: (*expr).clone(), options: *options, }) - .collect::>() + .collect() } // Convert each inner tuple to PhysicalSortExpr @@ -2190,7 +2148,7 @@ mod tests { expr: (*expr).clone(), options: *options, }) - .collect::>() + .collect() } // Convert each inner tuple to PhysicalSortExpr From 96f7d3ae676682c4fa9aa12f34fdf0e42f5c8e0b Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 11 Dec 2023 18:24:56 +0300 Subject: [PATCH 44/44] Update join suffix implementation --- datafusion/physical-expr/src/equivalence.rs | 105 +++++++++++++++++++- 1 file changed, 103 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index a02eeeeddc0d..b6e1164f41b0 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -702,8 +702,21 @@ impl OrderingEquivalenceClass { // Append orderings in `other` to all existing orderings in this equivalence // class. pub fn join_suffix(mut self, other: &Self) -> Self { - for ordering in other.iter() { - for idx in 0..self.orderings.len() { + let n_ordering = self.orderings.len(); + // Replicate entries before cross product + let n_cross = std::cmp::max(n_ordering, other.len() * n_ordering); + self.orderings = self + .orderings + .iter() + .cloned() + .cycle() + .take(n_cross) + .collect(); + // Suffix orderings of other to the current orderings. + for (outer_idx, ordering) in other.iter().enumerate() { + for idx in 0..n_ordering { + // Calculate cross product index + let idx = outer_idx * n_ordering + idx; self.orderings[idx].extend(ordering.iter().cloned()); } } @@ -5223,4 +5236,92 @@ mod tests { assert!(is_constant_recurse(&constants, &expr)); Ok(()) } + + #[test] + fn test_join_equivalence_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 offset = schema.fields.len(); + let col_a2 = &add_offset_to_expr(col_a.clone(), offset); + let col_b2 = &add_offset_to_expr(col_b.clone(), offset); + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let test_cases = vec![ + // ------- TEST CASE 1 -------- + // [a ASC], [b ASC] + ( + // [a ASC], [b ASC] + vec![vec![(col_a, option_asc)], vec![(col_b, option_asc)]], + // [a ASC], [b ASC] + vec![vec![(col_a, option_asc)], vec![(col_b, option_asc)]], + // expected [a ASC, a2 ASC], [a ASC, b2 ASC], [b ASC, a2 ASC], [b ASC, b2 ASC] + vec![ + vec![(col_a, option_asc), (col_a2, option_asc)], + vec![(col_a, option_asc), (col_b2, option_asc)], + vec![(col_b, option_asc), (col_a2, option_asc)], + vec![(col_b, option_asc), (col_b2, option_asc)], + ], + ), + // ------- TEST CASE 2 -------- + // [a ASC], [b ASC] + ( + // [a ASC], [b ASC], [c ASC] + vec![ + vec![(col_a, option_asc)], + vec![(col_b, option_asc)], + vec![(col_c, option_asc)], + ], + // [a ASC], [b ASC] + vec![vec![(col_a, option_asc)], vec![(col_b, option_asc)]], + // expected [a ASC, a2 ASC], [a ASC, b2 ASC], [b ASC, a2 ASC], [b ASC, b2 ASC], [c ASC, a2 ASC], [c ASC, b2 ASC] + vec![ + vec![(col_a, option_asc), (col_a2, option_asc)], + vec![(col_a, option_asc), (col_b2, option_asc)], + vec![(col_b, option_asc), (col_a2, option_asc)], + vec![(col_b, option_asc), (col_b2, option_asc)], + vec![(col_c, option_asc), (col_a2, option_asc)], + vec![(col_c, option_asc), (col_b2, option_asc)], + ], + ), + ]; + for (left_orderings, right_orderings, expected) in test_cases { + let mut left_eq_properties = EquivalenceProperties::new(schema.clone()); + let mut right_eq_properties = EquivalenceProperties::new(schema.clone()); + let left_orderings = convert_to_orderings(&left_orderings); + let right_orderings = convert_to_orderings(&right_orderings); + let expected = convert_to_orderings(&expected); + left_eq_properties.add_new_orderings(left_orderings); + right_eq_properties.add_new_orderings(right_orderings); + let join_eq = join_equivalence_properties( + left_eq_properties, + right_eq_properties, + &JoinType::Inner, + Arc::new(Schema::empty()), + &[true, false], + Some(JoinSide::Left), + &[], + ); + let orderings = &join_eq.oeq_class.orderings; + let err_msg = format!("expected: {:?}, actual:{:?}", expected, orderings); + assert_eq!( + join_eq.oeq_class.orderings.len(), + expected.len(), + "{}", + err_msg + ); + for ordering in orderings { + assert!( + expected.contains(ordering), + "{}, ordering: {:?}", + err_msg, + ordering + ); + } + } + Ok(()) + } }