From c61e7e597f80a3abd5015c996dc896a084112ab6 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Wed, 5 Mar 2025 08:17:19 +0800 Subject: [PATCH 01/33] Simplify Between expression to Eq (#14994) * simplify between equal case * cleanup --- .../simplify_expressions/expr_simplifier.rs | 21 ++++++++++++ .../src/simplify_expressions/utils.rs | 19 +++++++++++ .../sqllogictest/test_files/simplify_expr.slt | 34 +++++++++++++++++++ 3 files changed, 74 insertions(+) create mode 100644 datafusion/sqllogictest/test_files/simplify_expr.slt diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index e20580df4ce0..840c108905a9 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -942,6 +942,27 @@ impl TreeNodeRewriter for Simplifier<'_, S> { op: And, right, }) if is_op_with(Or, &left, &right) => Transformed::yes(*right), + // A >= constant AND constant <= A --> A = constant + Expr::BinaryExpr(BinaryExpr { + left, + op: And, + right, + }) if can_reduce_to_equal_statement(&left, &right) => { + if let Expr::BinaryExpr(BinaryExpr { + left: left_left, + right: left_right, + .. + }) = *left + { + Transformed::yes(Expr::BinaryExpr(BinaryExpr { + left: left_left, + op: Eq, + right: left_right, + })) + } else { + return internal_err!("can_reduce_to_equal_statement should only be called with a BinaryExpr"); + } + } // // Rules for Multiply diff --git a/datafusion/optimizer/src/simplify_expressions/utils.rs b/datafusion/optimizer/src/simplify_expressions/utils.rs index c30c3631c193..cf182175e48e 100644 --- a/datafusion/optimizer/src/simplify_expressions/utils.rs +++ b/datafusion/optimizer/src/simplify_expressions/utils.rs @@ -214,6 +214,25 @@ pub fn is_op_with(target_op: Operator, haystack: &Expr, needle: &Expr) -> bool { matches!(haystack, Expr::BinaryExpr(BinaryExpr { left, op, right }) if op == &target_op && (needle == left.as_ref() || needle == right.as_ref()) && !needle.is_volatile()) } +pub fn can_reduce_to_equal_statement(haystack: &Expr, needle: &Expr) -> bool { + match (haystack, needle) { + // a >= constant and constant <= a => a = constant + ( + Expr::BinaryExpr(BinaryExpr { + left, + op: Operator::GtEq, + right, + }), + Expr::BinaryExpr(BinaryExpr { + left: n_left, + op: Operator::LtEq, + right: n_right, + }), + ) if left == n_left && right == n_right => true, + _ => false, + } +} + /// returns true if `not_expr` is !`expr` (not) pub fn is_not_of(not_expr: &Expr, expr: &Expr) -> bool { matches!(not_expr, Expr::Not(inner) if expr == inner.as_ref()) diff --git a/datafusion/sqllogictest/test_files/simplify_expr.slt b/datafusion/sqllogictest/test_files/simplify_expr.slt new file mode 100644 index 000000000000..d10e603ea5f3 --- /dev/null +++ b/datafusion/sqllogictest/test_files/simplify_expr.slt @@ -0,0 +1,34 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +statement count 0 +create table t(a int) as values (1); + +# test between simplification +query TT +explain select a from t where a BETWEEN 3 and 3 +---- +logical_plan +01)Filter: t.a = Int32(3) +02)--TableScan: t projection=[a] +physical_plan +01)CoalesceBatchesExec: target_batch_size=8192 +02)--FilterExec: a@0 = 3 +03)----DataSourceExec: partitions=1, partition_sizes=[1] + +statement count 0 +drop table t; From 5d08325165c1a7b32e5e35164919e83d46735e98 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Wed, 5 Mar 2025 09:44:42 +0800 Subject: [PATCH 02/33] Count wildcard alias (#14927) * fix alias * append the string * window count * add column * fmt * rm todo * fixed partitioned * fix test * update doc * Suggestion to reduce API surface area --------- Co-authored-by: Andrew Lamb --- .../tests/dataframe/dataframe_functions.rs | 6 +- datafusion/core/tests/dataframe/mod.rs | 306 ++++++++++++++++-- datafusion/functions-aggregate/src/count.rs | 51 ++- .../sqllogictest/test_files/subquery.slt | 38 +++ 4 files changed, 360 insertions(+), 41 deletions(-) diff --git a/datafusion/core/tests/dataframe/dataframe_functions.rs b/datafusion/core/tests/dataframe/dataframe_functions.rs index 28c0740ca76b..fec3ab786fce 100644 --- a/datafusion/core/tests/dataframe/dataframe_functions.rs +++ b/datafusion/core/tests/dataframe/dataframe_functions.rs @@ -1145,9 +1145,9 @@ async fn test_count_wildcard() -> Result<()> { .build() .unwrap(); - let expected = "Sort: count(Int64(1)) ASC NULLS LAST [count(Int64(1)):Int64]\ - \n Projection: count(Int64(1)) [count(Int64(1)):Int64]\ - \n Aggregate: groupBy=[[test.b]], aggr=[[count(Int64(1))]] [b:UInt32, count(Int64(1)):Int64]\ + let expected = "Sort: count(*) ASC NULLS LAST [count(*):Int64]\ + \n Projection: count(*) [count(*):Int64]\ + \n Aggregate: groupBy=[[test.b]], aggr=[[count(Int64(1)) AS count(*)]] [b:UInt32, count(*):Int64]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; let formatted_plan = plan.display_indent_schema().to_string(); diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 1875180d503b..43428d6846a1 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -32,8 +32,7 @@ use arrow::datatypes::{ }; use arrow::error::ArrowError; use arrow::util::pretty::pretty_format_batches; -use datafusion_expr::utils::COUNT_STAR_EXPANSION; -use datafusion_functions_aggregate::count::{count_all, count_udaf}; +use datafusion_functions_aggregate::count::{count_all, count_all_window}; use datafusion_functions_aggregate::expr_fn::{ array_agg, avg, count, count_distinct, max, median, min, sum, }; @@ -2455,7 +2454,7 @@ async fn test_count_wildcard_on_sort() -> Result<()> { let ctx = create_join_context()?; let sql_results = ctx - .sql("select b,count(1) from t1 group by b order by count(1)") + .sql("select b, count(*) from t1 group by b order by count(*)") .await? .explain(false, false)? .collect() @@ -2469,9 +2468,52 @@ async fn test_count_wildcard_on_sort() -> Result<()> { .explain(false, false)? .collect() .await?; - //make sure sql plan same with df plan + + let expected_sql_result = "+---------------+------------------------------------------------------------------------------------------------------------+\ + \n| plan_type | plan |\ + \n+---------------+------------------------------------------------------------------------------------------------------------+\ + \n| logical_plan | Projection: t1.b, count(*) |\ + \n| | Sort: count(Int64(1)) AS count(*) AS count(*) ASC NULLS LAST |\ + \n| | Projection: t1.b, count(Int64(1)) AS count(*), count(Int64(1)) |\ + \n| | Aggregate: groupBy=[[t1.b]], aggr=[[count(Int64(1))]] |\ + \n| | TableScan: t1 projection=[b] |\ + \n| physical_plan | ProjectionExec: expr=[b@0 as b, count(*)@1 as count(*)] |\ + \n| | SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] |\ + \n| | SortExec: expr=[count(Int64(1))@2 ASC NULLS LAST], preserve_partitioning=[true] |\ + \n| | ProjectionExec: expr=[b@0 as b, count(Int64(1))@1 as count(*), count(Int64(1))@1 as count(Int64(1))] |\ + \n| | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(Int64(1))] |\ + \n| | CoalesceBatchesExec: target_batch_size=8192 |\ + \n| | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 |\ + \n| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |\ + \n| | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(Int64(1))] |\ + \n| | DataSourceExec: partitions=1, partition_sizes=[1] |\ + \n| | |\ + \n+---------------+------------------------------------------------------------------------------------------------------------+"; + assert_eq!( - pretty_format_batches(&sql_results)?.to_string(), + expected_sql_result, + pretty_format_batches(&sql_results)?.to_string() + ); + + let expected_df_result = "+---------------+--------------------------------------------------------------------------------+\ +\n| plan_type | plan |\ +\n+---------------+--------------------------------------------------------------------------------+\ +\n| logical_plan | Sort: count(*) ASC NULLS LAST |\ +\n| | Aggregate: groupBy=[[t1.b]], aggr=[[count(Int64(1)) AS count(*)]] |\ +\n| | TableScan: t1 projection=[b] |\ +\n| physical_plan | SortPreservingMergeExec: [count(*)@1 ASC NULLS LAST] |\ +\n| | SortExec: expr=[count(*)@1 ASC NULLS LAST], preserve_partitioning=[true] |\ +\n| | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(*)] |\ +\n| | CoalesceBatchesExec: target_batch_size=8192 |\ +\n| | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 |\ +\n| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |\ +\n| | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(*)] |\ +\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\ +\n| | |\ +\n+---------------+--------------------------------------------------------------------------------+"; + + assert_eq!( + expected_df_result, pretty_format_batches(&df_results)?.to_string() ); Ok(()) @@ -2481,12 +2523,35 @@ async fn test_count_wildcard_on_sort() -> Result<()> { async fn test_count_wildcard_on_where_in() -> Result<()> { let ctx = create_join_context()?; let sql_results = ctx - .sql("SELECT a,b FROM t1 WHERE a in (SELECT count(1) FROM t2)") + .sql("SELECT a, b FROM t1 WHERE a in (SELECT count(*) FROM t2)") .await? .explain(false, false)? .collect() .await?; + let expected_sql_result = "+---------------+------------------------------------------------------------------------------------------------------------------------+\ +\n| plan_type | plan |\ +\n+---------------+------------------------------------------------------------------------------------------------------------------------+\ +\n| logical_plan | LeftSemi Join: CAST(t1.a AS Int64) = __correlated_sq_1.count(*) |\ +\n| | TableScan: t1 projection=[a, b] |\ +\n| | SubqueryAlias: __correlated_sq_1 |\ +\n| | Projection: count(Int64(1)) AS count(*) |\ +\n| | Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] |\ +\n| | TableScan: t2 projection=[] |\ +\n| physical_plan | CoalesceBatchesExec: target_batch_size=8192 |\ +\n| | HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(count(*)@0, CAST(t1.a AS Int64)@2)], projection=[a@0, b@1] |\ +\n| | ProjectionExec: expr=[4 as count(*)] |\ +\n| | PlaceholderRowExec |\ +\n| | ProjectionExec: expr=[a@0 as a, b@1 as b, CAST(a@0 AS Int64) as CAST(t1.a AS Int64)] |\ +\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\ +\n| | |\ +\n+---------------+------------------------------------------------------------------------------------------------------------------------+"; + + assert_eq!( + expected_sql_result, + pretty_format_batches(&sql_results)?.to_string() + ); + // In the same SessionContext, AliasGenerator will increase subquery_alias id by 1 // https://github.com/apache/datafusion/blame/cf45eb9020092943b96653d70fafb143cc362e19/datafusion/optimizer/src/alias.rs#L40-L43 // for compare difference between sql and df logical plan, we need to create a new SessionContext here @@ -2509,9 +2574,26 @@ async fn test_count_wildcard_on_where_in() -> Result<()> { .collect() .await?; + let actual_df_result= "+---------------+------------------------------------------------------------------------------------------------------------------------+\ +\n| plan_type | plan |\ +\n+---------------+------------------------------------------------------------------------------------------------------------------------+\ +\n| logical_plan | LeftSemi Join: CAST(t1.a AS Int64) = __correlated_sq_1.count(*) |\ +\n| | TableScan: t1 projection=[a, b] |\ +\n| | SubqueryAlias: __correlated_sq_1 |\ +\n| | Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]] |\ +\n| | TableScan: t2 projection=[] |\ +\n| physical_plan | CoalesceBatchesExec: target_batch_size=8192 |\ +\n| | HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(count(*)@0, CAST(t1.a AS Int64)@2)], projection=[a@0, b@1] |\ +\n| | ProjectionExec: expr=[4 as count(*)] |\ +\n| | PlaceholderRowExec |\ +\n| | ProjectionExec: expr=[a@0 as a, b@1 as b, CAST(a@0 AS Int64) as CAST(t1.a AS Int64)] |\ +\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\ +\n| | |\ +\n+---------------+------------------------------------------------------------------------------------------------------------------------+"; + // make sure sql plan same with df plan assert_eq!( - pretty_format_batches(&sql_results)?.to_string(), + actual_df_result, pretty_format_batches(&df_results)?.to_string() ); @@ -2522,11 +2604,34 @@ async fn test_count_wildcard_on_where_in() -> Result<()> { async fn test_count_wildcard_on_where_exist() -> Result<()> { let ctx = create_join_context()?; let sql_results = ctx - .sql("SELECT a, b FROM t1 WHERE EXISTS (SELECT count(1) FROM t2)") + .sql("SELECT a, b FROM t1 WHERE EXISTS (SELECT count(*) FROM t2)") .await? .explain(false, false)? .collect() .await?; + + let actual_sql_result = + "+---------------+---------------------------------------------------------+\ + \n| plan_type | plan |\ + \n+---------------+---------------------------------------------------------+\ + \n| logical_plan | LeftSemi Join: |\ + \n| | TableScan: t1 projection=[a, b] |\ + \n| | SubqueryAlias: __correlated_sq_1 |\ + \n| | Projection: |\ + \n| | Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] |\ + \n| | TableScan: t2 projection=[] |\ + \n| physical_plan | NestedLoopJoinExec: join_type=RightSemi |\ + \n| | ProjectionExec: expr=[] |\ + \n| | PlaceholderRowExec |\ + \n| | DataSourceExec: partitions=1, partition_sizes=[1] |\ + \n| | |\ + \n+---------------+---------------------------------------------------------+"; + + assert_eq!( + actual_sql_result, + pretty_format_batches(&sql_results)?.to_string() + ); + let df_results = ctx .table("t1") .await? @@ -2545,9 +2650,24 @@ async fn test_count_wildcard_on_where_exist() -> Result<()> { .collect() .await?; - //make sure sql plan same with df plan + let actual_df_result = "+---------------+---------------------------------------------------------------------+\ + \n| plan_type | plan |\ + \n+---------------+---------------------------------------------------------------------+\ + \n| logical_plan | LeftSemi Join: |\ + \n| | TableScan: t1 projection=[a, b] |\ + \n| | SubqueryAlias: __correlated_sq_1 |\ + \n| | Projection: |\ + \n| | Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]] |\ + \n| | TableScan: t2 projection=[] |\ + \n| physical_plan | NestedLoopJoinExec: join_type=RightSemi |\ + \n| | ProjectionExec: expr=[] |\ + \n| | PlaceholderRowExec |\ + \n| | DataSourceExec: partitions=1, partition_sizes=[1] |\ + \n| | |\ + \n+---------------+---------------------------------------------------------------------+"; + assert_eq!( - pretty_format_batches(&sql_results)?.to_string(), + actual_df_result, pretty_format_batches(&df_results)?.to_string() ); @@ -2559,34 +2679,62 @@ async fn test_count_wildcard_on_window() -> Result<()> { let ctx = create_join_context()?; let sql_results = ctx - .sql("select count(1) OVER(ORDER BY a DESC RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING) from t1") + .sql("select count(*) OVER(ORDER BY a DESC RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING) from t1") .await? .explain(false, false)? .collect() .await?; + + let actual_sql_result = "+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\ +\n| plan_type | plan |\ +\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\ +\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\ +\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\ +\n| | TableScan: t1 projection=[a] |\ +\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(*) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\ +\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\ +\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\ +\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\ +\n| | |\ +\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"; + + assert_eq!( + actual_sql_result, + pretty_format_batches(&sql_results)?.to_string() + ); + let df_results = ctx .table("t1") .await? - .select(vec![Expr::WindowFunction(WindowFunction::new( - WindowFunctionDefinition::AggregateUDF(count_udaf()), - vec![Expr::Literal(COUNT_STAR_EXPANSION)], - )) - .order_by(vec![Sort::new(col("a"), false, true)]) - .window_frame(WindowFrame::new_bounds( - WindowFrameUnits::Range, - WindowFrameBound::Preceding(ScalarValue::UInt32(Some(6))), - WindowFrameBound::Following(ScalarValue::UInt32(Some(2))), - )) - .build() - .unwrap()])? + .select(vec![count_all_window() + .order_by(vec![Sort::new(col("a"), false, true)]) + .window_frame(WindowFrame::new_bounds( + WindowFrameUnits::Range, + WindowFrameBound::Preceding(ScalarValue::UInt32(Some(6))), + WindowFrameBound::Following(ScalarValue::UInt32(Some(2))), + )) + .build() + .unwrap()])? .explain(false, false)? .collect() .await?; - //make sure sql plan same with df plan + let actual_df_result = "+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\ +\n| plan_type | plan |\ +\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+\ +\n| logical_plan | Projection: count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING |\ +\n| | WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] |\ +\n| | TableScan: t1 projection=[a] |\ +\n| physical_plan | ProjectionExec: expr=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@1 as count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING] |\ +\n| | BoundedWindowAggExec: wdw=[count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: \"count(Int64(1)) ORDER BY [t1.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(6)), end_bound: Following(UInt32(2)), is_causal: false }], mode=[Sorted] |\ +\n| | SortExec: expr=[a@0 DESC], preserve_partitioning=[false] |\ +\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\ +\n| | |\ +\n+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+"; + assert_eq!( - pretty_format_batches(&df_results)?.to_string(), - pretty_format_batches(&sql_results)?.to_string() + actual_df_result, + pretty_format_batches(&df_results)?.to_string() ); Ok(()) @@ -2598,12 +2746,28 @@ async fn test_count_wildcard_on_aggregate() -> Result<()> { register_alltypes_tiny_pages_parquet(&ctx).await?; let sql_results = ctx - .sql("select count(1) from t1") + .sql("select count(*) from t1") .await? .explain(false, false)? .collect() .await?; + let actual_sql_result = + "+---------------+-----------------------------------------------------+\ +\n| plan_type | plan |\ +\n+---------------+-----------------------------------------------------+\ +\n| logical_plan | Projection: count(Int64(1)) AS count(*) |\ +\n| | Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] |\ +\n| | TableScan: t1 projection=[] |\ +\n| physical_plan | ProjectionExec: expr=[4 as count(*)] |\ +\n| | PlaceholderRowExec |\ +\n| | |\ +\n+---------------+-----------------------------------------------------+"; + assert_eq!( + actual_sql_result, + pretty_format_batches(&sql_results)?.to_string() + ); + // add `.select(vec![count_wildcard()])?` to make sure we can analyze all node instead of just top node. let df_results = ctx .table("t1") @@ -2614,9 +2778,17 @@ async fn test_count_wildcard_on_aggregate() -> Result<()> { .collect() .await?; - //make sure sql plan same with df plan + let actual_df_result = "+---------------+---------------------------------------------------------------+\ +\n| plan_type | plan |\ +\n+---------------+---------------------------------------------------------------+\ +\n| logical_plan | Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]] |\ +\n| | TableScan: t1 projection=[] |\ +\n| physical_plan | ProjectionExec: expr=[4 as count(*)] |\ +\n| | PlaceholderRowExec |\ +\n| | |\ +\n+---------------+---------------------------------------------------------------+"; assert_eq!( - pretty_format_batches(&sql_results)?.to_string(), + actual_df_result, pretty_format_batches(&df_results)?.to_string() ); @@ -2628,16 +2800,51 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { let ctx = create_join_context()?; let sql_results = ctx - .sql("select a,b from t1 where (select count(1) from t2 where t1.a = t2.a)>0;") + .sql("select a,b from t1 where (select count(*) from t2 where t1.a = t2.a)>0;") .await? .explain(false, false)? .collect() .await?; + let actual_sql_result = "+---------------+---------------------------------------------------------------------------------------------------------------------------+\ +\n| plan_type | plan |\ +\n+---------------+---------------------------------------------------------------------------------------------------------------------------+\ +\n| logical_plan | Projection: t1.a, t1.b |\ +\n| | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) |\ +\n| | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true |\ +\n| | Left Join: t1.a = __scalar_sq_1.a |\ +\n| | TableScan: t1 projection=[a, b] |\ +\n| | SubqueryAlias: __scalar_sq_1 |\ +\n| | Projection: count(Int64(1)) AS count(*), t2.a, Boolean(true) AS __always_true |\ +\n| | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1))]] |\ +\n| | TableScan: t2 projection=[a] |\ +\n| physical_plan | CoalesceBatchesExec: target_batch_size=8192 |\ +\n| | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] |\ +\n| | CoalesceBatchesExec: target_batch_size=8192 |\ +\n| | HashJoinExec: mode=Partitioned, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] |\ +\n| | CoalesceBatchesExec: target_batch_size=8192 |\ +\n| | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 |\ +\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\ +\n| | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] |\ +\n| | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] |\ +\n| | CoalesceBatchesExec: target_batch_size=8192 |\ +\n| | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 |\ +\n| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |\ +\n| | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] |\ +\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\ +\n| | |\ +\n+---------------+---------------------------------------------------------------------------------------------------------------------------+"; + assert_eq!( + actual_sql_result, + pretty_format_batches(&sql_results)?.to_string() + ); + // In the same SessionContext, AliasGenerator will increase subquery_alias id by 1 // https://github.com/apache/datafusion/blame/cf45eb9020092943b96653d70fafb143cc362e19/datafusion/optimizer/src/alias.rs#L40-L43 // for compare difference between sql and df logical plan, we need to create a new SessionContext here let ctx = create_join_context()?; + let agg_expr = count_all(); + let agg_expr_col = col(agg_expr.schema_name().to_string()); let df_results = ctx .table("t1") .await? @@ -2646,8 +2853,8 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { ctx.table("t2") .await? .filter(out_ref_col(DataType::UInt32, "t1.a").eq(col("t2.a")))? - .aggregate(vec![], vec![count_all()])? - .select(vec![col(count_all().to_string())])? + .aggregate(vec![], vec![agg_expr])? + .select(vec![agg_expr_col])? .into_unoptimized_plan(), )) .gt(lit(ScalarValue::UInt8(Some(0)))), @@ -2657,9 +2864,36 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { .collect() .await?; - //make sure sql plan same with df plan + let actual_df_result = "+---------------+---------------------------------------------------------------------------------------------------------------------------+\ +\n| plan_type | plan |\ +\n+---------------+---------------------------------------------------------------------------------------------------------------------------+\ +\n| logical_plan | Projection: t1.a, t1.b |\ +\n| | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) |\ +\n| | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true |\ +\n| | Left Join: t1.a = __scalar_sq_1.a |\ +\n| | TableScan: t1 projection=[a, b] |\ +\n| | SubqueryAlias: __scalar_sq_1 |\ +\n| | Projection: count(*), t2.a, Boolean(true) AS __always_true |\ +\n| | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1)) AS count(*)]] |\ +\n| | TableScan: t2 projection=[a] |\ +\n| physical_plan | CoalesceBatchesExec: target_batch_size=8192 |\ +\n| | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] |\ +\n| | CoalesceBatchesExec: target_batch_size=8192 |\ +\n| | HashJoinExec: mode=Partitioned, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] |\ +\n| | CoalesceBatchesExec: target_batch_size=8192 |\ +\n| | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 |\ +\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\ +\n| | ProjectionExec: expr=[count(*)@1 as count(*), a@0 as a, true as __always_true] |\ +\n| | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(*)] |\ +\n| | CoalesceBatchesExec: target_batch_size=8192 |\ +\n| | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 |\ +\n| | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 |\ +\n| | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(*)] |\ +\n| | DataSourceExec: partitions=1, partition_sizes=[1] |\ +\n| | |\ +\n+---------------+---------------------------------------------------------------------------------------------------------------------------+"; assert_eq!( - pretty_format_batches(&sql_results)?.to_string(), + actual_df_result, pretty_format_batches(&df_results)?.to_string() ); @@ -4228,7 +4462,9 @@ fn create_join_context() -> Result { ], )?; - let ctx = SessionContext::new(); + let config = SessionConfig::new().with_target_partitions(4); + let ctx = SessionContext::new_with_config(config); + // let ctx = SessionContext::new(); ctx.register_batch("t1", batch1)?; ctx.register_batch("t2", batch2)?; diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index a3339f0fceb9..2d995b4a4179 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -17,6 +17,7 @@ use ahash::RandomState; use datafusion_common::stats::Precision; +use datafusion_expr::expr::WindowFunction; use datafusion_functions_aggregate_common::aggregate::count_distinct::BytesViewDistinctCountAccumulator; use datafusion_macros::user_doc; use datafusion_physical_expr::expressions; @@ -51,7 +52,9 @@ use datafusion_expr::{ function::AccumulatorArgs, utils::format_state_name, Accumulator, AggregateUDFImpl, Documentation, EmitTo, GroupsAccumulator, SetMonotonicity, Signature, Volatility, }; -use datafusion_expr::{Expr, ReversedUDAF, StatisticsArgs, TypeSignature}; +use datafusion_expr::{ + Expr, ReversedUDAF, StatisticsArgs, TypeSignature, WindowFunctionDefinition, +}; use datafusion_functions_aggregate_common::aggregate::count_distinct::{ BytesDistinctCountAccumulator, FloatDistinctCountAccumulator, PrimitiveDistinctCountAccumulator, @@ -79,9 +82,51 @@ pub fn count_distinct(expr: Expr) -> Expr { )) } -/// Creates aggregation to count all rows, equivalent to `COUNT(*)`, `COUNT()`, `COUNT(1)` +/// Creates aggregation to count all rows. +/// +/// In SQL this is `SELECT COUNT(*) ... ` +/// +/// The expression is equivalent to `COUNT(*)`, `COUNT()`, `COUNT(1)`, and is +/// aliased to a column named `"count(*)"` for backward compatibility. +/// +/// Example +/// ``` +/// # use datafusion_functions_aggregate::count::count_all; +/// # use datafusion_expr::col; +/// // create `count(*)` expression +/// let expr = count_all(); +/// assert_eq!(expr.schema_name().to_string(), "count(*)"); +/// // if you need to refer to this column, use the `schema_name` function +/// let expr = col(expr.schema_name().to_string()); +/// ``` pub fn count_all() -> Expr { - count(Expr::Literal(COUNT_STAR_EXPANSION)) + count(Expr::Literal(COUNT_STAR_EXPANSION)).alias("count(*)") +} + +/// Creates window aggregation to count all rows. +/// +/// In SQL this is `SELECT COUNT(*) OVER (..) ... ` +/// +/// The expression is equivalent to `COUNT(*)`, `COUNT()`, `COUNT(1)` +/// +/// Example +/// ``` +/// # use datafusion_functions_aggregate::count::count_all_window; +/// # use datafusion_expr::col; +/// // create `count(*)` OVER ... window function expression +/// let expr = count_all_window(); +/// assert_eq!( +/// expr.schema_name().to_string(), +/// "count(Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING" +/// ); +/// // if you need to refer to this column, use the `schema_name` function +/// let expr = col(expr.schema_name().to_string()); +/// ``` +pub fn count_all_window() -> Expr { + Expr::WindowFunction(WindowFunction::new( + WindowFunctionDefinition::AggregateUDF(count_udaf()), + vec![Expr::Literal(COUNT_STAR_EXPANSION)], + )) } #[user_doc( diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 94c9eaf810fb..207bb72fd549 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -1393,3 +1393,41 @@ item1 1970-01-01T00:00:03 75 statement ok drop table source_table; + +statement count 0 +drop table t1; + +statement count 0 +drop table t2; + +statement count 0 +drop table t3; + +# test count wildcard +statement count 0 +create table t1(a int) as values (1); + +statement count 0 +create table t2(b int) as values (1); + +query I +SELECT a FROM t1 WHERE EXISTS (SELECT count(*) FROM t2) +---- +1 + +query TT +explain SELECT a FROM t1 WHERE EXISTS (SELECT count(*) FROM t2) +---- +logical_plan +01)LeftSemi Join: +02)--TableScan: t1 projection=[a] +03)--SubqueryAlias: __correlated_sq_1 +04)----Projection: +05)------Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] +06)--------TableScan: t2 projection=[] + +statement count 0 +drop table t1; + +statement count 0 +drop table t2; From 75977692c12bda72301ccf65067532c5135fbd5c Mon Sep 17 00:00:00 2001 From: zjregee Date: Wed, 5 Mar 2025 10:06:05 +0800 Subject: [PATCH 03/33] replace TypeSignature::String with TypeSignature::Coercible (#14917) * deprecated use of TypeSignature::String * make kernel functions private --- datafusion/functions-nested/src/string.rs | 17 +- datafusion/functions/src/regex/regexplike.rs | 27 +++- datafusion/functions/src/string/bit_length.rs | 17 +- datafusion/functions/src/string/contains.rs | 85 ++++++---- datafusion/functions/src/string/ends_with.rs | 43 ++++- .../functions/src/string/levenshtein.rs | 150 +++++++++++------- datafusion/functions/src/string/lower.rs | 14 +- .../functions/src/string/octet_length.rs | 17 +- datafusion/functions/src/string/replace.rs | 81 ++++++++-- datafusion/functions/src/string/upper.rs | 14 +- datafusion/functions/src/unicode/initcap.rs | 14 +- datafusion/functions/src/unicode/strpos.rs | 22 ++- datafusion/sqllogictest/test_files/scalar.slt | 2 +- .../test_files/string/string_view.slt | 14 +- 14 files changed, 378 insertions(+), 139 deletions(-) diff --git a/datafusion/functions-nested/src/string.rs b/datafusion/functions-nested/src/string.rs index 99af3e95c804..d60d1a6e4de0 100644 --- a/datafusion/functions-nested/src/string.rs +++ b/datafusion/functions-nested/src/string.rs @@ -24,7 +24,6 @@ use arrow::array::{ UInt8Array, }; use arrow::datatypes::{DataType, Field}; -use datafusion_expr::TypeSignature; use datafusion_common::{ internal_datafusion_err, not_impl_err, plan_err, DataFusionError, Result, @@ -44,8 +43,10 @@ use arrow::datatypes::DataType::{ }; use datafusion_common::cast::{as_large_list_array, as_list_array}; use datafusion_common::exec_err; +use datafusion_common::types::logical_string; use datafusion_expr::{ - ColumnarValue, Documentation, ScalarUDFImpl, Signature, Volatility, + Coercion, ColumnarValue, Documentation, ScalarUDFImpl, Signature, TypeSignature, + TypeSignatureClass, Volatility, }; use datafusion_functions::{downcast_arg, downcast_named_arg}; use datafusion_macros::user_doc; @@ -251,7 +252,17 @@ impl StringToArray { pub fn new() -> Self { Self { signature: Signature::one_of( - vec![TypeSignature::String(2), TypeSignature::String(3)], + vec![ + TypeSignature::Coercible(vec![ + Coercion::new_exact(TypeSignatureClass::Native(logical_string())), + Coercion::new_exact(TypeSignatureClass::Native(logical_string())), + ]), + TypeSignature::Coercible(vec![ + Coercion::new_exact(TypeSignatureClass::Native(logical_string())), + Coercion::new_exact(TypeSignatureClass::Native(logical_string())), + Coercion::new_exact(TypeSignatureClass::Native(logical_string())), + ]), + ], Volatility::Immutable, ), aliases: vec![String::from("string_to_list")], diff --git a/datafusion/functions/src/regex/regexplike.rs b/datafusion/functions/src/regex/regexplike.rs index 6006309306d5..2080bb9fe818 100644 --- a/datafusion/functions/src/regex/regexplike.rs +++ b/datafusion/functions/src/regex/regexplike.rs @@ -21,12 +21,15 @@ use arrow::array::{Array, ArrayRef, AsArray, GenericStringArray}; use arrow::compute::kernels::regexp; use arrow::datatypes::DataType; use arrow::datatypes::DataType::{LargeUtf8, Utf8, Utf8View}; -use datafusion_common::exec_err; -use datafusion_common::ScalarValue; -use datafusion_common::{arrow_datafusion_err, plan_err}; -use datafusion_common::{internal_err, DataFusionError, Result}; -use datafusion_expr::{ColumnarValue, Documentation, TypeSignature}; -use datafusion_expr::{ScalarUDFImpl, Signature, Volatility}; +use datafusion_common::types::logical_string; +use datafusion_common::{ + arrow_datafusion_err, exec_err, internal_err, plan_err, DataFusionError, Result, + ScalarValue, +}; +use datafusion_expr::{ + Coercion, ColumnarValue, Documentation, ScalarUDFImpl, Signature, TypeSignature, + TypeSignatureClass, Volatility, +}; use datafusion_macros::user_doc; use std::any::Any; @@ -79,7 +82,17 @@ impl RegexpLikeFunc { pub fn new() -> Self { Self { signature: Signature::one_of( - vec![TypeSignature::String(2), TypeSignature::String(3)], + vec![ + TypeSignature::Coercible(vec![ + Coercion::new_exact(TypeSignatureClass::Native(logical_string())), + Coercion::new_exact(TypeSignatureClass::Native(logical_string())), + ]), + TypeSignature::Coercible(vec![ + Coercion::new_exact(TypeSignatureClass::Native(logical_string())), + Coercion::new_exact(TypeSignatureClass::Native(logical_string())), + Coercion::new_exact(TypeSignatureClass::Native(logical_string())), + ]), + ], Volatility::Immutable, ), } diff --git a/datafusion/functions/src/string/bit_length.rs b/datafusion/functions/src/string/bit_length.rs index 2a782c59963e..f8740aa4178b 100644 --- a/datafusion/functions/src/string/bit_length.rs +++ b/datafusion/functions/src/string/bit_length.rs @@ -20,9 +20,13 @@ use arrow::datatypes::DataType; use std::any::Any; use crate::utils::utf8_to_int_type; -use datafusion_common::{utils::take_function_args, Result, ScalarValue}; -use datafusion_expr::{ColumnarValue, Documentation, Volatility}; -use datafusion_expr::{ScalarFunctionArgs, ScalarUDFImpl, Signature}; +use datafusion_common::types::logical_string; +use datafusion_common::utils::take_function_args; +use datafusion_common::{Result, ScalarValue}; +use datafusion_expr::{ + Coercion, ColumnarValue, Documentation, ScalarFunctionArgs, ScalarUDFImpl, Signature, + TypeSignatureClass, Volatility, +}; use datafusion_macros::user_doc; #[user_doc( @@ -55,7 +59,12 @@ impl Default for BitLengthFunc { impl BitLengthFunc { pub fn new() -> Self { Self { - signature: Signature::string(1, Volatility::Immutable), + signature: Signature::coercible( + vec![Coercion::new_exact(TypeSignatureClass::Native( + logical_string(), + ))], + Volatility::Immutable, + ), } } } diff --git a/datafusion/functions/src/string/contains.rs b/datafusion/functions/src/string/contains.rs index 77774cdb5e1d..05a3edf61c5a 100644 --- a/datafusion/functions/src/string/contains.rs +++ b/datafusion/functions/src/string/contains.rs @@ -20,12 +20,12 @@ use arrow::array::{Array, ArrayRef, AsArray}; use arrow::compute::contains as arrow_contains; use arrow::datatypes::DataType; use arrow::datatypes::DataType::{Boolean, LargeUtf8, Utf8, Utf8View}; -use datafusion_common::exec_err; -use datafusion_common::DataFusionError; -use datafusion_common::Result; +use datafusion_common::types::logical_string; +use datafusion_common::{exec_err, DataFusionError, Result}; +use datafusion_expr::binary::{binary_to_string_coercion, string_coercion}; use datafusion_expr::{ - ColumnarValue, Documentation, ScalarFunctionArgs, ScalarUDFImpl, Signature, - Volatility, + Coercion, ColumnarValue, Documentation, ScalarFunctionArgs, ScalarUDFImpl, Signature, + TypeSignatureClass, Volatility, }; use datafusion_macros::user_doc; use std::any::Any; @@ -60,7 +60,13 @@ impl Default for ContainsFunc { impl ContainsFunc { pub fn new() -> Self { Self { - signature: Signature::string(2, Volatility::Immutable), + signature: Signature::coercible( + vec![ + Coercion::new_exact(TypeSignatureClass::Native(logical_string())), + Coercion::new_exact(TypeSignatureClass::Native(logical_string())), + ], + Volatility::Immutable, + ), } } } @@ -92,29 +98,52 @@ impl ScalarUDFImpl for ContainsFunc { } /// use `arrow::compute::contains` to do the calculation for contains -pub fn contains(args: &[ArrayRef]) -> Result { - match (args[0].data_type(), args[1].data_type()) { - (Utf8View, Utf8View) => { - let mod_str = args[0].as_string_view(); - let match_str = args[1].as_string_view(); - let res = arrow_contains(mod_str, match_str)?; - Ok(Arc::new(res) as ArrayRef) - } - (Utf8, Utf8) => { - let mod_str = args[0].as_string::(); - let match_str = args[1].as_string::(); - let res = arrow_contains(mod_str, match_str)?; - Ok(Arc::new(res) as ArrayRef) - } - (LargeUtf8, LargeUtf8) => { - let mod_str = args[0].as_string::(); - let match_str = args[1].as_string::(); - let res = arrow_contains(mod_str, match_str)?; - Ok(Arc::new(res) as ArrayRef) - } - other => { - exec_err!("Unsupported data type {other:?} for function `contains`.") +fn contains(args: &[ArrayRef]) -> Result { + if let Some(coercion_data_type) = + string_coercion(args[0].data_type(), args[1].data_type()).or_else(|| { + binary_to_string_coercion(args[0].data_type(), args[1].data_type()) + }) + { + let arg0 = if args[0].data_type() == &coercion_data_type { + Arc::clone(&args[0]) + } else { + arrow::compute::kernels::cast::cast(&args[0], &coercion_data_type)? + }; + let arg1 = if args[1].data_type() == &coercion_data_type { + Arc::clone(&args[1]) + } else { + arrow::compute::kernels::cast::cast(&args[1], &coercion_data_type)? + }; + + match coercion_data_type { + Utf8View => { + let mod_str = arg0.as_string_view(); + let match_str = arg1.as_string_view(); + let res = arrow_contains(mod_str, match_str)?; + Ok(Arc::new(res) as ArrayRef) + } + Utf8 => { + let mod_str = arg0.as_string::(); + let match_str = arg1.as_string::(); + let res = arrow_contains(mod_str, match_str)?; + Ok(Arc::new(res) as ArrayRef) + } + LargeUtf8 => { + let mod_str = arg0.as_string::(); + let match_str = arg1.as_string::(); + let res = arrow_contains(mod_str, match_str)?; + Ok(Arc::new(res) as ArrayRef) + } + other => { + exec_err!("Unsupported data type {other:?} for function `contains`.") + } } + } else { + exec_err!( + "Unsupported data type {:?}, {:?} for function `contains`.", + args[0].data_type(), + args[1].data_type() + ) } } diff --git a/datafusion/functions/src/string/ends_with.rs b/datafusion/functions/src/string/ends_with.rs index 5cca79de14ff..eafc310236ee 100644 --- a/datafusion/functions/src/string/ends_with.rs +++ b/datafusion/functions/src/string/ends_with.rs @@ -22,9 +22,13 @@ use arrow::array::ArrayRef; use arrow::datatypes::DataType; use crate::utils::make_scalar_function; +use datafusion_common::types::logical_string; use datafusion_common::{internal_err, Result}; -use datafusion_expr::{ColumnarValue, Documentation, Volatility}; -use datafusion_expr::{ScalarFunctionArgs, ScalarUDFImpl, Signature}; +use datafusion_expr::binary::{binary_to_string_coercion, string_coercion}; +use datafusion_expr::{ + Coercion, ColumnarValue, Documentation, ScalarFunctionArgs, ScalarUDFImpl, Signature, + TypeSignatureClass, Volatility, +}; use datafusion_macros::user_doc; #[user_doc( @@ -62,7 +66,13 @@ impl Default for EndsWithFunc { impl EndsWithFunc { pub fn new() -> Self { Self { - signature: Signature::string(2, Volatility::Immutable), + signature: Signature::coercible( + vec![ + Coercion::new_exact(TypeSignatureClass::Native(logical_string())), + Coercion::new_exact(TypeSignatureClass::Native(logical_string())), + ], + Volatility::Immutable, + ), } } } @@ -102,10 +112,29 @@ impl ScalarUDFImpl for EndsWithFunc { /// Returns true if string ends with suffix. /// ends_with('alphabet', 'abet') = 't' -pub fn ends_with(args: &[ArrayRef]) -> Result { - let result = arrow::compute::kernels::comparison::ends_with(&args[0], &args[1])?; - - Ok(Arc::new(result) as ArrayRef) +fn ends_with(args: &[ArrayRef]) -> Result { + if let Some(coercion_data_type) = + string_coercion(args[0].data_type(), args[1].data_type()).or_else(|| { + binary_to_string_coercion(args[0].data_type(), args[1].data_type()) + }) + { + let arg0 = if args[0].data_type() == &coercion_data_type { + Arc::clone(&args[0]) + } else { + arrow::compute::kernels::cast::cast(&args[0], &coercion_data_type)? + }; + let arg1 = if args[1].data_type() == &coercion_data_type { + Arc::clone(&args[1]) + } else { + arrow::compute::kernels::cast::cast(&args[1], &coercion_data_type)? + }; + let result = arrow::compute::kernels::comparison::ends_with(&arg0, &arg1)?; + Ok(Arc::new(result) as ArrayRef) + } else { + internal_err!( + "Unsupported data types for ends_with. Expected Utf8, LargeUtf8 or Utf8View" + ) + } } #[cfg(test)] diff --git a/datafusion/functions/src/string/levenshtein.rs b/datafusion/functions/src/string/levenshtein.rs index a19fcc5b476c..a1a486c7d3cf 100644 --- a/datafusion/functions/src/string/levenshtein.rs +++ b/datafusion/functions/src/string/levenshtein.rs @@ -23,10 +23,17 @@ use arrow::datatypes::DataType; use crate::utils::{make_scalar_function, utf8_to_int_type}; use datafusion_common::cast::{as_generic_string_array, as_string_view_array}; +use datafusion_common::types::logical_string; use datafusion_common::utils::datafusion_strsim; -use datafusion_common::{exec_err, utils::take_function_args, Result}; -use datafusion_expr::{ColumnarValue, Documentation}; -use datafusion_expr::{ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility}; +use datafusion_common::utils::take_function_args; +use datafusion_common::{exec_err, Result}; +use datafusion_expr::type_coercion::binary::{ + binary_to_string_coercion, string_coercion, +}; +use datafusion_expr::{ + Coercion, ColumnarValue, Documentation, ScalarFunctionArgs, ScalarUDFImpl, Signature, + TypeSignatureClass, Volatility, +}; use datafusion_macros::user_doc; #[user_doc( @@ -64,7 +71,13 @@ impl Default for LevenshteinFunc { impl LevenshteinFunc { pub fn new() -> Self { Self { - signature: Signature::string(2, Volatility::Immutable), + signature: Signature::coercible( + vec![ + Coercion::new_exact(TypeSignatureClass::Native(logical_string())), + Coercion::new_exact(TypeSignatureClass::Native(logical_string())), + ], + Volatility::Immutable, + ), } } } @@ -83,7 +96,13 @@ impl ScalarUDFImpl for LevenshteinFunc { } fn return_type(&self, arg_types: &[DataType]) -> Result { - utf8_to_int_type(&arg_types[0], "levenshtein") + if let Some(coercion_data_type) = string_coercion(&arg_types[0], &arg_types[1]) + .or_else(|| binary_to_string_coercion(&arg_types[0], &arg_types[1])) + { + utf8_to_int_type(&coercion_data_type, "levenshtein") + } else { + exec_err!("Unsupported data types for levenshtein. Expected Utf8, LargeUtf8 or Utf8View") + } } fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result { @@ -107,60 +126,79 @@ impl ScalarUDFImpl for LevenshteinFunc { ///Returns the Levenshtein distance between the two given strings. /// LEVENSHTEIN('kitten', 'sitting') = 3 -pub fn levenshtein(args: &[ArrayRef]) -> Result { +fn levenshtein(args: &[ArrayRef]) -> Result { let [str1, str2] = take_function_args("levenshtein", args)?; - match str1.data_type() { - DataType::Utf8View => { - let str1_array = as_string_view_array(&str1)?; - let str2_array = as_string_view_array(&str2)?; - let result = str1_array - .iter() - .zip(str2_array.iter()) - .map(|(string1, string2)| match (string1, string2) { - (Some(string1), Some(string2)) => { - Some(datafusion_strsim::levenshtein(string1, string2) as i32) - } - _ => None, - }) - .collect::(); - Ok(Arc::new(result) as ArrayRef) - } - DataType::Utf8 => { - let str1_array = as_generic_string_array::(&str1)?; - let str2_array = as_generic_string_array::(&str2)?; - let result = str1_array - .iter() - .zip(str2_array.iter()) - .map(|(string1, string2)| match (string1, string2) { - (Some(string1), Some(string2)) => { - Some(datafusion_strsim::levenshtein(string1, string2) as i32) - } - _ => None, - }) - .collect::(); - Ok(Arc::new(result) as ArrayRef) - } - DataType::LargeUtf8 => { - let str1_array = as_generic_string_array::(&str1)?; - let str2_array = as_generic_string_array::(&str2)?; - let result = str1_array - .iter() - .zip(str2_array.iter()) - .map(|(string1, string2)| match (string1, string2) { - (Some(string1), Some(string2)) => { - Some(datafusion_strsim::levenshtein(string1, string2) as i64) - } - _ => None, - }) - .collect::(); - Ok(Arc::new(result) as ArrayRef) - } - other => { - exec_err!( - "levenshtein was called with {other} datatype arguments. It requires Utf8View, Utf8 or LargeUtf8." - ) + if let Some(coercion_data_type) = + string_coercion(args[0].data_type(), args[1].data_type()).or_else(|| { + binary_to_string_coercion(args[0].data_type(), args[1].data_type()) + }) + { + let str1 = if str1.data_type() == &coercion_data_type { + Arc::clone(str1) + } else { + arrow::compute::kernels::cast::cast(&str1, &coercion_data_type)? + }; + let str2 = if str2.data_type() == &coercion_data_type { + Arc::clone(str2) + } else { + arrow::compute::kernels::cast::cast(&str2, &coercion_data_type)? + }; + + match coercion_data_type { + DataType::Utf8View => { + let str1_array = as_string_view_array(&str1)?; + let str2_array = as_string_view_array(&str2)?; + let result = str1_array + .iter() + .zip(str2_array.iter()) + .map(|(string1, string2)| match (string1, string2) { + (Some(string1), Some(string2)) => { + Some(datafusion_strsim::levenshtein(string1, string2) as i32) + } + _ => None, + }) + .collect::(); + Ok(Arc::new(result) as ArrayRef) + } + DataType::Utf8 => { + let str1_array = as_generic_string_array::(&str1)?; + let str2_array = as_generic_string_array::(&str2)?; + let result = str1_array + .iter() + .zip(str2_array.iter()) + .map(|(string1, string2)| match (string1, string2) { + (Some(string1), Some(string2)) => { + Some(datafusion_strsim::levenshtein(string1, string2) as i32) + } + _ => None, + }) + .collect::(); + Ok(Arc::new(result) as ArrayRef) + } + DataType::LargeUtf8 => { + let str1_array = as_generic_string_array::(&str1)?; + let str2_array = as_generic_string_array::(&str2)?; + let result = str1_array + .iter() + .zip(str2_array.iter()) + .map(|(string1, string2)| match (string1, string2) { + (Some(string1), Some(string2)) => { + Some(datafusion_strsim::levenshtein(string1, string2) as i64) + } + _ => None, + }) + .collect::(); + Ok(Arc::new(result) as ArrayRef) + } + other => { + exec_err!( + "levenshtein was called with {other} datatype arguments. It requires Utf8View, Utf8 or LargeUtf8." + ) + } } + } else { + exec_err!("Unsupported data types for levenshtein. Expected Utf8, LargeUtf8 or Utf8View") } } diff --git a/datafusion/functions/src/string/lower.rs b/datafusion/functions/src/string/lower.rs index 375717e23d6d..226275b13999 100644 --- a/datafusion/functions/src/string/lower.rs +++ b/datafusion/functions/src/string/lower.rs @@ -20,9 +20,12 @@ use std::any::Any; use crate::string::common::to_lower; use crate::utils::utf8_to_str_type; +use datafusion_common::types::logical_string; use datafusion_common::Result; -use datafusion_expr::{ColumnarValue, Documentation}; -use datafusion_expr::{ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility}; +use datafusion_expr::{ + Coercion, ColumnarValue, Documentation, ScalarFunctionArgs, ScalarUDFImpl, Signature, + TypeSignatureClass, Volatility, +}; use datafusion_macros::user_doc; #[user_doc( @@ -55,7 +58,12 @@ impl Default for LowerFunc { impl LowerFunc { pub fn new() -> Self { Self { - signature: Signature::string(1, Volatility::Immutable), + signature: Signature::coercible( + vec![Coercion::new_exact(TypeSignatureClass::Native( + logical_string(), + ))], + Volatility::Immutable, + ), } } } diff --git a/datafusion/functions/src/string/octet_length.rs b/datafusion/functions/src/string/octet_length.rs index 46175c96cdc6..17ea2726b071 100644 --- a/datafusion/functions/src/string/octet_length.rs +++ b/datafusion/functions/src/string/octet_length.rs @@ -20,9 +20,13 @@ use arrow::datatypes::DataType; use std::any::Any; use crate::utils::utf8_to_int_type; -use datafusion_common::{utils::take_function_args, Result, ScalarValue}; -use datafusion_expr::{ColumnarValue, Documentation, Volatility}; -use datafusion_expr::{ScalarFunctionArgs, ScalarUDFImpl, Signature}; +use datafusion_common::types::logical_string; +use datafusion_common::utils::take_function_args; +use datafusion_common::{Result, ScalarValue}; +use datafusion_expr::{ + Coercion, ColumnarValue, Documentation, ScalarFunctionArgs, ScalarUDFImpl, Signature, + TypeSignatureClass, Volatility, +}; use datafusion_macros::user_doc; #[user_doc( @@ -55,7 +59,12 @@ impl Default for OctetLengthFunc { impl OctetLengthFunc { pub fn new() -> Self { Self { - signature: Signature::string(1, Volatility::Immutable), + signature: Signature::coercible( + vec![Coercion::new_exact(TypeSignatureClass::Native( + logical_string(), + ))], + Volatility::Immutable, + ), } } } diff --git a/datafusion/functions/src/string/replace.rs b/datafusion/functions/src/string/replace.rs index a3488b561fd2..de70215c49c7 100644 --- a/datafusion/functions/src/string/replace.rs +++ b/datafusion/functions/src/string/replace.rs @@ -23,9 +23,15 @@ use arrow::datatypes::DataType; use crate::utils::{make_scalar_function, utf8_to_str_type}; use datafusion_common::cast::{as_generic_string_array, as_string_view_array}; +use datafusion_common::types::logical_string; use datafusion_common::{exec_err, Result}; -use datafusion_expr::{ColumnarValue, Documentation, Volatility}; -use datafusion_expr::{ScalarFunctionArgs, ScalarUDFImpl, Signature}; +use datafusion_expr::type_coercion::binary::{ + binary_to_string_coercion, string_coercion, +}; +use datafusion_expr::{ + Coercion, ColumnarValue, Documentation, ScalarFunctionArgs, ScalarUDFImpl, Signature, + TypeSignatureClass, Volatility, +}; use datafusion_macros::user_doc; #[user_doc( doc_section(label = "String Functions"), @@ -60,7 +66,14 @@ impl Default for ReplaceFunc { impl ReplaceFunc { pub fn new() -> Self { Self { - signature: Signature::string(3, Volatility::Immutable), + signature: Signature::coercible( + vec![ + Coercion::new_exact(TypeSignatureClass::Native(logical_string())), + Coercion::new_exact(TypeSignatureClass::Native(logical_string())), + Coercion::new_exact(TypeSignatureClass::Native(logical_string())), + ], + Volatility::Immutable, + ), } } } @@ -79,19 +92,64 @@ impl ScalarUDFImpl for ReplaceFunc { } fn return_type(&self, arg_types: &[DataType]) -> Result { - utf8_to_str_type(&arg_types[0], "replace") + if let Some(coercion_data_type) = string_coercion(&arg_types[0], &arg_types[1]) + .and_then(|dt| string_coercion(&dt, &arg_types[2])) + .or_else(|| { + binary_to_string_coercion(&arg_types[0], &arg_types[1]) + .and_then(|dt| binary_to_string_coercion(&dt, &arg_types[2])) + }) + { + utf8_to_str_type(&coercion_data_type, "replace") + } else { + exec_err!("Unsupported data types for replace. Expected Utf8, LargeUtf8 or Utf8View") + } } fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result { - match args.args[0].data_type() { - DataType::Utf8 => make_scalar_function(replace::, vec![])(&args.args), - DataType::LargeUtf8 => { - make_scalar_function(replace::, vec![])(&args.args) + let data_types = args + .args + .iter() + .map(|arg| arg.data_type()) + .collect::>(); + + if let Some(coercion_type) = string_coercion(&data_types[0], &data_types[1]) + .and_then(|dt| string_coercion(&dt, &data_types[2])) + .or_else(|| { + binary_to_string_coercion(&data_types[0], &data_types[1]) + .and_then(|dt| binary_to_string_coercion(&dt, &data_types[2])) + }) + { + let mut converted_args = Vec::with_capacity(args.args.len()); + for arg in &args.args { + if arg.data_type() == coercion_type { + converted_args.push(arg.clone()); + } else { + let converted = arg.cast_to(&coercion_type, None)?; + converted_args.push(converted); + } } - DataType::Utf8View => make_scalar_function(replace_view, vec![])(&args.args), - other => { - exec_err!("Unsupported data type {other:?} for function replace") + + match coercion_type { + DataType::Utf8 => { + make_scalar_function(replace::, vec![])(&converted_args) + } + DataType::LargeUtf8 => { + make_scalar_function(replace::, vec![])(&converted_args) + } + DataType::Utf8View => { + make_scalar_function(replace_view, vec![])(&converted_args) + } + other => exec_err!( + "Unsupported coercion data type {other:?} for function replace" + ), } + } else { + exec_err!( + "Unsupported data type {:?}, {:?}, {:?} for function replace.", + data_types[0], + data_types[1], + data_types[2] + ) } } @@ -117,6 +175,7 @@ fn replace_view(args: &[ArrayRef]) -> Result { Ok(Arc::new(result) as ArrayRef) } + /// Replaces all occurrences in string of substring from with substring to. /// replace('abcdefabcdef', 'cd', 'XX') = 'abXXefabXXef' fn replace(args: &[ArrayRef]) -> Result { diff --git a/datafusion/functions/src/string/upper.rs b/datafusion/functions/src/string/upper.rs index d27b54d29bc6..2fec7305d183 100644 --- a/datafusion/functions/src/string/upper.rs +++ b/datafusion/functions/src/string/upper.rs @@ -18,9 +18,12 @@ use crate::string::common::to_upper; use crate::utils::utf8_to_str_type; use arrow::datatypes::DataType; +use datafusion_common::types::logical_string; use datafusion_common::Result; -use datafusion_expr::{ColumnarValue, Documentation}; -use datafusion_expr::{ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility}; +use datafusion_expr::{ + Coercion, ColumnarValue, Documentation, ScalarFunctionArgs, ScalarUDFImpl, Signature, + TypeSignatureClass, Volatility, +}; use datafusion_macros::user_doc; use std::any::Any; @@ -54,7 +57,12 @@ impl Default for UpperFunc { impl UpperFunc { pub fn new() -> Self { Self { - signature: Signature::string(1, Volatility::Immutable), + signature: Signature::coercible( + vec![Coercion::new_exact(TypeSignatureClass::Native( + logical_string(), + ))], + Volatility::Immutable, + ), } } } diff --git a/datafusion/functions/src/unicode/initcap.rs b/datafusion/functions/src/unicode/initcap.rs index a8a4dd0fa249..c9b0cb77b096 100644 --- a/datafusion/functions/src/unicode/initcap.rs +++ b/datafusion/functions/src/unicode/initcap.rs @@ -25,9 +25,12 @@ use arrow::datatypes::DataType; use crate::utils::{make_scalar_function, utf8_to_str_type}; use datafusion_common::cast::{as_generic_string_array, as_string_view_array}; +use datafusion_common::types::logical_string; use datafusion_common::{exec_err, Result}; -use datafusion_expr::{ColumnarValue, Documentation, Volatility}; -use datafusion_expr::{ScalarUDFImpl, Signature}; +use datafusion_expr::{ + Coercion, ColumnarValue, Documentation, ScalarUDFImpl, Signature, TypeSignatureClass, + Volatility, +}; use datafusion_macros::user_doc; #[user_doc( @@ -61,7 +64,12 @@ impl Default for InitcapFunc { impl InitcapFunc { pub fn new() -> Self { Self { - signature: Signature::string(1, Volatility::Immutable), + signature: Signature::coercible( + vec![Coercion::new_exact(TypeSignatureClass::Native( + logical_string(), + ))], + Volatility::Immutable, + ), } } } diff --git a/datafusion/functions/src/unicode/strpos.rs b/datafusion/functions/src/unicode/strpos.rs index a0ad2e3f0b75..b3bc73a29585 100644 --- a/datafusion/functions/src/unicode/strpos.rs +++ b/datafusion/functions/src/unicode/strpos.rs @@ -23,9 +23,11 @@ use arrow::array::{ ArrayRef, ArrowPrimitiveType, AsArray, PrimitiveArray, StringArrayType, }; use arrow::datatypes::{ArrowNativeType, DataType, Int32Type, Int64Type}; +use datafusion_common::types::logical_string; use datafusion_common::{exec_err, internal_err, Result}; use datafusion_expr::{ - ColumnarValue, Documentation, ScalarUDFImpl, Signature, Volatility, + Coercion, ColumnarValue, Documentation, ScalarUDFImpl, Signature, TypeSignatureClass, + Volatility, }; use datafusion_macros::user_doc; @@ -60,7 +62,13 @@ impl Default for StrposFunc { impl StrposFunc { pub fn new() -> Self { Self { - signature: Signature::string(2, Volatility::Immutable), + signature: Signature::coercible( + vec![ + Coercion::new_exact(TypeSignatureClass::Native(logical_string())), + Coercion::new_exact(TypeSignatureClass::Native(logical_string())), + ], + Volatility::Immutable, + ), aliases: vec![String::from("instr"), String::from("position")], } } @@ -115,6 +123,11 @@ fn strpos(args: &[ArrayRef]) -> Result { let substring_array = args[1].as_string::(); calculate_strpos::<_, _, Int32Type>(string_array, substring_array) } + (DataType::Utf8, DataType::Utf8View) => { + let string_array = args[0].as_string::(); + let substring_array = args[1].as_string_view(); + calculate_strpos::<_, _, Int32Type>(string_array, substring_array) + } (DataType::Utf8, DataType::LargeUtf8) => { let string_array = args[0].as_string::(); let substring_array = args[1].as_string::(); @@ -125,6 +138,11 @@ fn strpos(args: &[ArrayRef]) -> Result { let substring_array = args[1].as_string::(); calculate_strpos::<_, _, Int64Type>(string_array, substring_array) } + (DataType::LargeUtf8, DataType::Utf8View) => { + let string_array = args[0].as_string::(); + let substring_array = args[1].as_string_view(); + calculate_strpos::<_, _, Int64Type>(string_array, substring_array) + } (DataType::LargeUtf8, DataType::LargeUtf8) => { let string_array = args[0].as_string::(); let substring_array = args[1].as_string::(); diff --git a/datafusion/sqllogictest/test_files/scalar.slt b/datafusion/sqllogictest/test_files/scalar.slt index 66413775b393..f583d659fd4f 100644 --- a/datafusion/sqllogictest/test_files/scalar.slt +++ b/datafusion/sqllogictest/test_files/scalar.slt @@ -1927,7 +1927,7 @@ select position('' in '') ---- 1 -query error DataFusion error: Error during planning: Function 'strpos' expects NativeType::String but received NativeType::Int64 +query error DataFusion error: Error during planning: Internal error: Expect TypeSignatureClass::Native\(LogicalType\(Native\(String\), String\)\) but received NativeType::Int64, DataType: Int64 select position(1 in 1) query I diff --git a/datafusion/sqllogictest/test_files/string/string_view.slt b/datafusion/sqllogictest/test_files/string/string_view.slt index 754937e18f14..69c4b9bfcb4b 100644 --- a/datafusion/sqllogictest/test_files/string/string_view.slt +++ b/datafusion/sqllogictest/test_files/string/string_view.slt @@ -660,7 +660,7 @@ EXPLAIN SELECT FROM test; ---- logical_plan -01)Projection: contains(test.column1_utf8view, Utf8View("foo")) AS c1, contains(test.column1_utf8view, test.column2_utf8view) AS c2, contains(test.column1_utf8view, CAST(test.column2_large_utf8 AS Utf8View)) AS c3, contains(CAST(test.column1_utf8 AS Utf8View), test.column2_utf8view) AS c4, contains(test.column1_utf8, test.column2_utf8) AS c5, contains(CAST(test.column1_utf8 AS LargeUtf8), test.column2_large_utf8) AS c6, contains(CAST(test.column1_large_utf8 AS Utf8View), test.column1_utf8view) AS c7, contains(test.column1_large_utf8, CAST(test.column2_utf8 AS LargeUtf8)) AS c8, contains(test.column1_large_utf8, test.column2_large_utf8) AS c9 +01)Projection: contains(test.column1_utf8view, Utf8("foo")) AS c1, contains(test.column1_utf8view, test.column2_utf8view) AS c2, contains(test.column1_utf8view, test.column2_large_utf8) AS c3, contains(test.column1_utf8, test.column2_utf8view) AS c4, contains(test.column1_utf8, test.column2_utf8) AS c5, contains(test.column1_utf8, test.column2_large_utf8) AS c6, contains(test.column1_large_utf8, test.column1_utf8view) AS c7, contains(test.column1_large_utf8, test.column2_utf8) AS c8, contains(test.column1_large_utf8, test.column2_large_utf8) AS c9 02)--TableScan: test projection=[column1_utf8, column2_utf8, column1_large_utf8, column2_large_utf8, column1_utf8view, column2_utf8view] ## Ensure no casts for ENDS_WITH @@ -671,7 +671,7 @@ EXPLAIN SELECT FROM test; ---- logical_plan -01)Projection: ends_with(test.column1_utf8view, Utf8View("foo")) AS c1, ends_with(test.column2_utf8view, test.column2_utf8view) AS c2 +01)Projection: ends_with(test.column1_utf8view, Utf8("foo")) AS c1, ends_with(test.column2_utf8view, test.column2_utf8view) AS c2 02)--TableScan: test projection=[column1_utf8view, column2_utf8view] ## Ensure no casts for LEVENSHTEIN @@ -682,7 +682,7 @@ EXPLAIN SELECT FROM test; ---- logical_plan -01)Projection: levenshtein(test.column1_utf8view, Utf8View("foo")) AS c1, levenshtein(test.column1_utf8view, test.column2_utf8view) AS c2 +01)Projection: levenshtein(test.column1_utf8view, Utf8("foo")) AS c1, levenshtein(test.column1_utf8view, test.column2_utf8view) AS c2 02)--TableScan: test projection=[column1_utf8view, column2_utf8view] ## Ensure no casts for LOWER @@ -784,7 +784,7 @@ EXPLAIN SELECT FROM test; ---- logical_plan -01)Projection: regexp_like(test.column1_utf8view, Utf8View("^https?://(?:www\.)?([^/]+)/.*$")) AS k +01)Projection: regexp_like(test.column1_utf8view, Utf8("^https?://(?:www\.)?([^/]+)/.*$")) AS k 02)--TableScan: test projection=[column1_utf8view] ## Ensure no casts for REGEXP_MATCH @@ -825,7 +825,7 @@ EXPLAIN SELECT FROM test; ---- logical_plan -01)Projection: replace(test.column1_utf8view, Utf8View("foo"), Utf8View("bar")) AS c1, replace(test.column1_utf8view, test.column2_utf8view, Utf8View("bar")) AS c2 +01)Projection: replace(test.column1_utf8view, Utf8("foo"), Utf8("bar")) AS c1, replace(test.column1_utf8view, test.column2_utf8view, Utf8("bar")) AS c2 02)--TableScan: test projection=[column1_utf8view, column2_utf8view] ## Ensure no casts for REVERSE @@ -906,7 +906,7 @@ EXPLAIN SELECT FROM test; ---- logical_plan -01)Projection: strpos(test.column1_utf8view, Utf8View("f")) AS c, strpos(test.column1_utf8view, test.column2_utf8view) AS c2 +01)Projection: strpos(test.column1_utf8view, Utf8("f")) AS c, strpos(test.column1_utf8view, test.column2_utf8view) AS c2 02)--TableScan: test projection=[column1_utf8view, column2_utf8view] ## Ensure no casts for SUBSTR @@ -1078,7 +1078,7 @@ EXPLAIN SELECT FROM test; ---- logical_plan -01)Projection: string_to_array(test.column1_utf8view, Utf8View(",")) AS c +01)Projection: string_to_array(test.column1_utf8view, Utf8(",")) AS c 02)--TableScan: test projection=[column1_utf8view] ## Ensure no unexpected casts for array_to_string From ee2498ff7ab6202e023e8251c7b290db9200d9a5 Mon Sep 17 00:00:00 2001 From: wiedld Date: Wed, 5 Mar 2025 02:47:04 -0800 Subject: [PATCH 04/33] chore(15003): add identation to plans, to make easier to read (#15007) --- .../enforce_distribution.rs | 1171 +++++++++-------- 1 file changed, 590 insertions(+), 581 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index fc2394d889d1..85d826109f89 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -19,11 +19,11 @@ use std::fmt::Debug; use std::ops::Deref; use std::sync::Arc; +use crate::physical_optimizer::test_utils::parquet_exec_with_sort; use crate::physical_optimizer::test_utils::{ check_integrity, coalesce_partitions_exec, repartition_exec, schema, sort_merge_join_exec, sort_preserving_merge_exec, }; -use crate::physical_optimizer::test_utils::{parquet_exec_with_sort, trim_plan_display}; use arrow::compute::SortOptions; use datafusion::config::ConfigOptions; @@ -61,7 +61,9 @@ use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeE use datafusion_physical_plan::union::UnionExec; use datafusion_physical_plan::ExecutionPlanProperties; use datafusion_physical_plan::PlanProperties; -use datafusion_physical_plan::{displayable, DisplayAs, DisplayFormatType, Statistics}; +use datafusion_physical_plan::{ + get_plan_string, DisplayAs, DisplayFormatType, Statistics, +}; /// Models operators like BoundedWindowExec that require an input /// ordering but is easy to construct @@ -358,8 +360,7 @@ fn ensure_distribution_helper( macro_rules! plans_matches_expected { ($EXPECTED_LINES: expr, $PLAN: expr) => { let physical_plan = $PLAN; - let formatted = displayable(physical_plan.as_ref()).indent(true).to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); + let actual = get_plan_string(&physical_plan); let expected_plan_lines: Vec<&str> = $EXPECTED_LINES .iter().map(|s| *s).collect(); @@ -485,8 +486,7 @@ macro_rules! assert_optimized { let optimized = optimizer.optimize(optimized, &config)?; // Now format correctly - let plan = displayable(optimized.as_ref()).indent(true).to_string(); - let actual_lines = trim_plan_display(&plan); + let actual_lines = get_plan_string(&optimized); assert_eq!( &expected_lines, &actual_lines, @@ -500,8 +500,7 @@ macro_rules! assert_plan_txt { ($EXPECTED_LINES: expr, $PLAN: expr) => { let expected_lines: Vec<&str> = $EXPECTED_LINES.iter().map(|s| *s).collect(); // Now format correctly - let plan = displayable($PLAN.as_ref()).indent(true).to_string(); - let actual_lines = trim_plan_display(&plan); + let actual_lines = get_plan_string(&$PLAN); assert_eq!( &expected_lines, &actual_lines, @@ -542,9 +541,11 @@ fn multi_hash_joins() -> Result<()> { for join_type in join_types { let join = hash_join_exec(left.clone(), right.clone(), &join_on, &join_type); - let join_plan = format!( - "HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, b1@1)]" - ); + let join_plan = |shift| -> String { + format!("{}HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, b1@1)]", " ".repeat(shift)) + }; + let join_plan_indent2 = join_plan(2); + let join_plan_indent4 = join_plan(4); match join_type { JoinType::Inner @@ -572,33 +573,33 @@ fn multi_hash_joins() -> Result<()> { // Should include 3 RepartitionExecs JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => vec![ top_join_plan.as_str(), - join_plan.as_str(), - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + &join_plan_indent2, + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], // Should include 4 RepartitionExecs _ => vec![ top_join_plan.as_str(), - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - join_plan.as_str(), - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + &join_plan_indent4, + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], }; assert_optimized!(expected, top_join.clone(), true); @@ -635,34 +636,34 @@ fn multi_hash_joins() -> Result<()> { JoinType::Inner | JoinType::Right | JoinType::RightSemi | JoinType::RightAnti => vec![ top_join_plan.as_str(), - join_plan.as_str(), - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + &join_plan_indent2, + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], // Should include 4 RepartitionExecs _ => vec![ top_join_plan.as_str(), - "RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10", - join_plan.as_str(), - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10", + &join_plan_indent4, + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], }; assert_optimized!(expected, top_join.clone(), true); @@ -710,17 +711,17 @@ fn multi_joins_after_alias() -> Result<()> { // Output partition need to respect the Alias and should not introduce additional RepartitionExec let expected = &[ "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a1@0, c@2)]", - "ProjectionExec: expr=[a@0 as a1, a@0 as a2]", - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " ProjectionExec: expr=[a@0 as a1, a@0 as a2]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, top_join.clone(), true); assert_optimized!(expected, top_join, false); @@ -736,17 +737,17 @@ fn multi_joins_after_alias() -> Result<()> { // Output partition need to respect the Alias and should not introduce additional RepartitionExec let expected = &[ "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a2@1, c@2)]", - "ProjectionExec: expr=[a@0 as a1, a@0 as a2]", - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " ProjectionExec: expr=[a@0 as a1, a@0 as a2]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, top_join.clone(), true); assert_optimized!(expected, top_join, false); @@ -787,19 +788,19 @@ fn multi_joins_after_multi_alias() -> Result<()> { // The original Output partition can not satisfy the Join requirements and need to add an additional RepartitionExec let expected = &[ "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, c@2)]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "ProjectionExec: expr=[c1@0 as a]", - "ProjectionExec: expr=[c@2 as c1]", - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + " ProjectionExec: expr=[c1@0 as a]", + " ProjectionExec: expr=[c@2 as c1]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, top_join.clone(), true); @@ -831,16 +832,16 @@ fn join_after_agg_alias() -> Result<()> { // Only two RepartitionExecs added let expected = &[ "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a1@0, a2@0)]", - "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", - "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[]", - "RepartitionExec: partitioning=Hash([a2@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a2], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", + " RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", + " AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[]", + " RepartitionExec: partitioning=Hash([a2@0], 10), input_partitions=10", + " AggregateExec: mode=Partial, gby=[a@0 as a2], aggr=[]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, join.clone(), true); assert_optimized!(expected, join, false); @@ -883,17 +884,17 @@ fn hash_join_key_ordering() -> Result<()> { // Only two RepartitionExecs added let expected = &[ "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b1@1, b@0), (a1@0, a@1)]", - "ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", - "AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[]", - "RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", + " AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[]", + " RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", + " AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", + " RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", + " AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, join.clone(), true); assert_optimized!(expected, join, false); @@ -1002,24 +1003,24 @@ fn multi_hash_join_key_ordering() -> Result<()> { // The bottom joins' join key ordering is adjusted based on the top join. And the top join should not introduce additional RepartitionExec let expected = &[ "FilterExec: c@6 > 1", - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(B@2, b1@6), (C@3, c@2), (AA@1, a1@5)]", - "ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C]", - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)]", - "RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)]", - "RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(B@2, b1@6), (C@3, c@2), (AA@1, a1@5)]", + " ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)]", + " RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)]", + " RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, filter_top_join.clone(), true); assert_optimized!(expected, filter_top_join, false); @@ -1141,23 +1142,23 @@ fn reorder_join_keys_to_left_input() -> Result<()> { // The top joins' join key ordering is adjusted based on the children inputs. let expected = &[ top_join_plan.as_str(), - "ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C]", - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1), (c@2, c1@2)]", - "RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([a1@0, b1@1, c1@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)]", - "RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1), (c@2, c1@2)]", + " RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([a1@0, b1@1, c1@2], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)]", + " RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_plan_txt!(expected, reordered); @@ -1275,23 +1276,23 @@ fn reorder_join_keys_to_right_input() -> Result<()> { // The top joins' join key ordering is adjusted based on the children inputs. let expected = &[ top_join_plan.as_str(), - "ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C]", - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1)]", - "RepartitionExec: partitioning=Hash([a@0, b@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([a1@0, b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)]", - "RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1)]", + " RepartitionExec: partitioning=Hash([a@0, b@1], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([a1@0, b1@1], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)]", + " RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_plan_txt!(expected, reordered); @@ -1331,7 +1332,15 @@ fn multi_smj_joins() -> Result<()> { for join_type in join_types { let join = sort_merge_join_exec(left.clone(), right.clone(), &join_on, &join_type); - let join_plan = format!("SortMergeJoin: join_type={join_type}, on=[(a@0, b1@1)]"); + let join_plan = |shift| -> String { + format!( + "{}SortMergeJoin: join_type={join_type}, on=[(a@0, b1@1)]", + " ".repeat(shift) + ) + }; + let join_plan_indent2 = join_plan(2); + let join_plan_indent6 = join_plan(6); + let join_plan_indent10 = join_plan(10); // Top join on (a == c) let top_join_on = vec![( @@ -1348,20 +1357,20 @@ fn multi_smj_joins() -> Result<()> { JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => vec![ top_join_plan.as_str(), - join_plan.as_str(), - "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + &join_plan_indent2, + " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], // Should include 7 RepartitionExecs (4 hash, 3 round-robin), 4 SortExecs // Since ordering of the left child is not preserved after SortMergeJoin @@ -1375,22 +1384,22 @@ fn multi_smj_joins() -> Result<()> { _ => vec![ top_join_plan.as_str(), // Below 2 operators are differences introduced, when join mode is changed - "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - join_plan.as_str(), - "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + &join_plan_indent6, + " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], }; assert_optimized!(expected, top_join.clone(), true, true); @@ -1400,20 +1409,20 @@ fn multi_smj_joins() -> Result<()> { JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => vec![ top_join_plan.as_str(), - join_plan.as_str(), - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + &join_plan_indent2, + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", + " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], // Should include 8 RepartitionExecs (4 hash, 8 round-robin), 4 SortExecs // Since ordering of the left child is not preserved after SortMergeJoin @@ -1427,24 +1436,24 @@ fn multi_smj_joins() -> Result<()> { _ => vec![ top_join_plan.as_str(), // Below 4 operators are differences introduced, when join mode is changed - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - join_plan.as_str(), - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " CoalescePartitionsExec", + &join_plan_indent10, + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", + " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], }; assert_optimized!(expected_first_sort_enforcement, top_join, false, true); @@ -1466,40 +1475,40 @@ fn multi_smj_joins() -> Result<()> { // Should include 6 RepartitionExecs(3 hash, 3 round-robin) and 3 SortExecs JoinType::Inner | JoinType::Right => vec![ top_join_plan.as_str(), - join_plan.as_str(), - "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + &join_plan_indent2, + " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], // Should include 7 RepartitionExecs (4 hash, 3 round-robin) and 4 SortExecs JoinType::Left | JoinType::Full => vec![ top_join_plan.as_str(), - "SortExec: expr=[b1@6 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10", - join_plan.as_str(), - "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " SortExec: expr=[b1@6 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10", + &join_plan_indent6, + " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], // this match arm cannot be reached _ => unreachable!() @@ -1510,42 +1519,42 @@ fn multi_smj_joins() -> Result<()> { // Should include 6 RepartitionExecs (3 of them preserves order) and 3 SortExecs JoinType::Inner | JoinType::Right => vec![ top_join_plan.as_str(), - join_plan.as_str(), - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + &join_plan_indent2, + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", + " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], // Should include 8 RepartitionExecs (4 of them preserves order) and 4 SortExecs JoinType::Left | JoinType::Full => vec![ top_join_plan.as_str(), - "RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@6 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[b1@6 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - join_plan.as_str(), - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@6 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortExec: expr=[b1@6 ASC], preserve_partitioning=[false]", + " CoalescePartitionsExec", + &join_plan_indent10, + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", + " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], // this match arm cannot be reached _ => unreachable!() @@ -1608,47 +1617,47 @@ fn smj_join_key_ordering() -> Result<()> { // Only two RepartitionExecs added let expected = &[ "SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)]", - "SortExec: expr=[b3@1 ASC, a3@0 ASC], preserve_partitioning=[true]", - "ProjectionExec: expr=[a1@0 as a3, b1@1 as b3]", - "ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", - "AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[]", - "RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[true]", - "ProjectionExec: expr=[a@1 as a2, b@0 as b2]", - "AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " SortExec: expr=[b3@1 ASC, a3@0 ASC], preserve_partitioning=[true]", + " ProjectionExec: expr=[a1@0 as a3, b1@1 as b3]", + " ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", + " AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[]", + " RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", + " AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[true]", + " ProjectionExec: expr=[a@1 as a2, b@0 as b2]", + " AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", + " RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", + " AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, join.clone(), true, true); let expected_first_sort_enforcement = &[ "SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)]", - "RepartitionExec: partitioning=Hash([b3@1, a3@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b3@1 ASC, a3@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[b3@1 ASC, a3@0 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "ProjectionExec: expr=[a1@0 as a3, b1@1 as b3]", - "ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", - "AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[]", - "RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "RepartitionExec: partitioning=Hash([b2@1, a2@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b2@1 ASC, a2@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "ProjectionExec: expr=[a@1 as a2, b@0 as b2]", - "AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([b3@1, a3@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b3@1 ASC, a3@0 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortExec: expr=[b3@1 ASC, a3@0 ASC], preserve_partitioning=[false]", + " CoalescePartitionsExec", + " ProjectionExec: expr=[a1@0 as a3, b1@1 as b3]", + " ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", + " AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[]", + " RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", + " AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([b2@1, a2@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b2@1 ASC, a2@0 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[false]", + " CoalescePartitionsExec", + " ProjectionExec: expr=[a@1 as a2, b@0 as b2]", + " AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", + " RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", + " AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected_first_sort_enforcement, join, false, true); @@ -1678,8 +1687,8 @@ fn merge_does_not_need_sort() -> Result<()> { // data is already sorted let expected = &[ "SortPreservingMergeExec: [a@0 ASC]", - "CoalesceBatchesExec: target_batch_size=4096", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", + " CoalesceBatchesExec: target_batch_size=4096", + " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; assert_optimized!(expected, exec, true); @@ -1689,9 +1698,9 @@ fn merge_does_not_need_sort() -> Result<()> { // SortExec at the top. let expected = &[ "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "CoalesceBatchesExec: target_batch_size=4096", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", + " CoalescePartitionsExec", + " CoalesceBatchesExec: target_batch_size=4096", + " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; assert_optimized!(expected, exec, false); @@ -1721,18 +1730,18 @@ fn union_to_interleave() -> Result<()> { // Only two RepartitionExecs added, no final RepartitionExec required let expected = &[ "AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[]", - "AggregateExec: mode=Partial, gby=[a1@0 as a2], aggr=[]", - "InterleaveExec", - "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", - "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", - "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " AggregateExec: mode=Partial, gby=[a1@0 as a2], aggr=[]", + " InterleaveExec", + " AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", + " RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", + " AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", + " RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", + " AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan.clone(), false); @@ -1763,19 +1772,19 @@ fn union_not_to_interleave() -> Result<()> { // Only two RepartitionExecs added, no final RepartitionExec required let expected = &[ "AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[]", - "RepartitionExec: partitioning=Hash([a2@0], 10), input_partitions=20", - "AggregateExec: mode=Partial, gby=[a1@0 as a2], aggr=[]", - "UnionExec", - "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", - "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", - "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([a2@0], 10), input_partitions=20", + " AggregateExec: mode=Partial, gby=[a1@0 as a2], aggr=[]", + " UnionExec", + " AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", + " RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", + " AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", + " RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", + " AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; // no sort in the plan but since we need it as a parameter, make it default false let prefer_existing_sort = false; @@ -1807,10 +1816,10 @@ fn added_repartition_to_single_partition() -> Result<()> { let expected = [ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -1825,11 +1834,11 @@ fn repartition_deepest_node() -> Result<()> { let expected = &[ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + " FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -1844,12 +1853,12 @@ fn repartition_unsorted_limit() -> Result<()> { let expected = &[ "GlobalLimitExec: skip=0, fetch=100", - "CoalescePartitionsExec", - "LocalLimitExec: fetch=100", - "FilterExec: c@2 = 0", + " CoalescePartitionsExec", + " LocalLimitExec: fetch=100", + " FilterExec: c@2 = 0", // nothing sorts the data, so the local limit doesn't require sorted data either - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); @@ -1869,10 +1878,10 @@ fn repartition_sorted_limit() -> Result<()> { let expected = &[ "GlobalLimitExec: skip=0, fetch=100", - "LocalLimitExec: fetch=100", + " LocalLimitExec: fetch=100", // data is sorted so can't repartition here - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -1894,12 +1903,12 @@ fn repartition_sorted_limit_with_filter() -> Result<()> { let expected = &[ "SortRequiredExec: [c@2 ASC]", - "FilterExec: c@2 = 0", + " FilterExec: c@2 = 0", // We can use repartition here, ordering requirement by SortRequiredExec // is still satisfied. - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); @@ -1918,19 +1927,19 @@ fn repartition_ignores_limit() -> Result<()> { let expected = &[ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "GlobalLimitExec: skip=0, fetch=100", - "CoalescePartitionsExec", - "LocalLimitExec: fetch=100", - "FilterExec: c@2 = 0", + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " GlobalLimitExec: skip=0, fetch=100", + " CoalescePartitionsExec", + " LocalLimitExec: fetch=100", + " FilterExec: c@2 = 0", // repartition should happen prior to the filter to maximize parallelism - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "GlobalLimitExec: skip=0, fetch=100", - "LocalLimitExec: fetch=100", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " GlobalLimitExec: skip=0, fetch=100", + " LocalLimitExec: fetch=100", // Expect no repartition to happen for local limit - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -1945,11 +1954,11 @@ fn repartition_ignores_union() -> Result<()> { let expected = &[ "UnionExec", // Expect no repartition of DataSourceExec - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); @@ -1971,7 +1980,7 @@ fn repartition_through_sort_preserving_merge() -> Result<()> { // need resort as the data was not sorted correctly let expected = &[ "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -1996,15 +2005,15 @@ fn repartition_ignores_sort_preserving_merge() -> Result<()> { // should not repartition, since increased parallelism is not beneficial for SortPReservingMerge let expected = &[ "SortPreservingMergeExec: [c@2 ASC]", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); let expected = &[ "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + " CoalescePartitionsExec", + " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, plan, false); @@ -2025,19 +2034,19 @@ fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { // should not repartition / sort (as the data was already sorted) let expected = &[ "SortPreservingMergeExec: [c@2 ASC]", - "UnionExec", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + " UnionExec", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); let expected = &[ "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "UnionExec", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + " CoalescePartitionsExec", + " UnionExec", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, plan, false); @@ -2061,9 +2070,9 @@ fn repartition_does_not_destroy_sort() -> Result<()> { // during repartitioning ordering is preserved let expected = &[ "SortRequiredExec: [d@3 ASC]", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet", + " FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true, true); @@ -2100,12 +2109,12 @@ fn repartition_does_not_destroy_sort_more_complex() -> Result<()> { let expected = &[ "UnionExec", // union input 1: no repartitioning - "SortRequiredExec: [c@2 ASC]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + " SortRequiredExec: [c@2 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", // union input 2: should repartition - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -2134,22 +2143,22 @@ fn repartition_transitively_with_projection() -> Result<()> { let expected = &[ "SortPreservingMergeExec: [sum@0 ASC]", - "SortExec: expr=[sum@0 ASC], preserve_partitioning=[true]", + " SortExec: expr=[sum@0 ASC], preserve_partitioning=[true]", // Since this projection is not trivial, increasing parallelism is beneficial - "ProjectionExec: expr=[a@0 + b@1 as sum]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " ProjectionExec: expr=[a@0 + b@1 as sum]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); let expected_first_sort_enforcement = &[ "SortExec: expr=[sum@0 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", + " CoalescePartitionsExec", // Since this projection is not trivial, increasing parallelism is beneficial - "ProjectionExec: expr=[a@0 + b@1 as sum]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " ProjectionExec: expr=[a@0 + b@1 as sum]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected_first_sort_enforcement, plan, false); @@ -2180,8 +2189,8 @@ fn repartition_ignores_transitively_with_projection() -> Result<()> { let expected = &[ "SortRequiredExec: [c@2 ASC]", // Since this projection is trivial, increasing parallelism is not beneficial - "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + " ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", + " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -2213,8 +2222,8 @@ fn repartition_transitively_past_sort_with_projection() -> Result<()> { let expected = &[ "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", // Since this projection is trivial, increasing parallelism is not beneficial - "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -2233,22 +2242,22 @@ fn repartition_transitively_past_sort_with_filter() -> Result<()> { let expected = &[ "SortPreservingMergeExec: [a@0 ASC]", - "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", // Expect repartition on the input to the sort (as it can benefit from additional parallelism) - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); let expected_first_sort_enforcement = &[ "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "FilterExec: c@2 = 0", + " CoalescePartitionsExec", + " FilterExec: c@2 = 0", // Expect repartition on the input of the filter (as it can benefit from additional parallelism) - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected_first_sort_enforcement, plan, false); @@ -2279,23 +2288,23 @@ fn repartition_transitively_past_sort_with_projection_and_filter() -> Result<()> let expected = &[ "SortPreservingMergeExec: [a@0 ASC]", // Expect repartition on the input to the sort (as it can benefit from additional parallelism) - "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", - "FilterExec: c@2 = 0", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + " ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", + " FilterExec: c@2 = 0", // repartition is lowest down - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); let expected_first_sort_enforcement = &[ "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " CoalescePartitionsExec", + " ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", + " FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected_first_sort_enforcement, plan, false); @@ -2310,15 +2319,15 @@ fn parallelization_single_partition() -> Result<()> { let expected_parquet = [ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", + " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + " DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=parquet", ]; let expected_csv = [ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + " RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", + " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + " DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true, false, 2, true, 10); assert_optimized!(expected_csv, plan_csv, true, false, 2, true, 10); @@ -2342,8 +2351,8 @@ fn parallelization_multiple_files() -> Result<()> { // https://github.com/apache/datafusion/issues/8451 let expected = [ "SortRequiredExec: [a@0 ASC]", - "FilterExec: c@2 = 0", - "DataSourceExec: file_groups={3 groups: [[x:0..50], [y:0..100], [x:50..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; + " FilterExec: c@2 = 0", + " DataSourceExec: file_groups={3 groups: [[x:0..50], [y:0..100], [x:50..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; let target_partitions = 3; let repartition_size = 1; assert_optimized!( @@ -2359,8 +2368,8 @@ fn parallelization_multiple_files() -> Result<()> { let expected = [ "SortRequiredExec: [a@0 ASC]", - "FilterExec: c@2 = 0", - "DataSourceExec: file_groups={8 groups: [[x:0..25], [y:0..25], [x:25..50], [y:25..50], [x:50..75], [y:50..75], [x:75..100], [y:75..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", + " FilterExec: c@2 = 0", + " DataSourceExec: file_groups={8 groups: [[x:0..25], [y:0..25], [x:25..50], [y:25..50], [x:50..75], [y:50..75], [x:75..100], [y:75..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; let target_partitions = 8; let repartition_size = 1; @@ -2392,17 +2401,17 @@ fn parallelization_compressed_csv() -> Result<()> { let expected_not_partitioned = [ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + " RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", + " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; let expected_partitioned = [ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + " RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", + " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + " DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; for compression_type in compression_types { @@ -2436,17 +2445,17 @@ fn parallelization_two_partitions() -> Result<()> { let expected_parquet = [ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + " RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", + " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", // Plan already has two partitions - "DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=parquet", + " DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=parquet", ]; let expected_csv = [ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + " RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", + " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", // Plan already has two partitions - "DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + " DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true, false, 2, true, 10); assert_optimized!(expected_csv, plan_csv, true, false, 2, true, 10); @@ -2461,17 +2470,17 @@ fn parallelization_two_partitions_into_four() -> Result<()> { let expected_parquet = [ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + " RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", + " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", // Multiple source files splitted across partitions - "DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=parquet", + " DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=parquet", ]; let expected_csv = [ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + " RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", + " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", // Multiple source files splitted across partitions - "DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + " DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true, false, 4, true, 10); assert_optimized!(expected_csv, plan_csv, true, false, 4, true, 10); @@ -2491,19 +2500,19 @@ fn parallelization_sorted_limit() -> Result<()> { let expected_parquet = &[ "GlobalLimitExec: skip=0, fetch=100", - "LocalLimitExec: fetch=100", + " LocalLimitExec: fetch=100", // data is sorted so can't repartition here - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", // Doesn't parallelize for SortExec without preserve_partitioning - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; let expected_csv = &[ "GlobalLimitExec: skip=0, fetch=100", - "LocalLimitExec: fetch=100", + " LocalLimitExec: fetch=100", // data is sorted so can't repartition here - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", // Doesn't parallelize for SortExec without preserve_partitioning - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true); assert_optimized!(expected_csv, plan_csv, true); @@ -2527,27 +2536,27 @@ fn parallelization_limit_with_filter() -> Result<()> { let expected_parquet = &[ "GlobalLimitExec: skip=0, fetch=100", - "CoalescePartitionsExec", - "LocalLimitExec: fetch=100", - "FilterExec: c@2 = 0", + " CoalescePartitionsExec", + " LocalLimitExec: fetch=100", + " FilterExec: c@2 = 0", // even though data is sorted, we can use repartition here. Since // ordering is not used in subsequent stages anyway. - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", // SortExec doesn't benefit from input partitioning - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; let expected_csv = &[ "GlobalLimitExec: skip=0, fetch=100", - "CoalescePartitionsExec", - "LocalLimitExec: fetch=100", - "FilterExec: c@2 = 0", + " CoalescePartitionsExec", + " LocalLimitExec: fetch=100", + " FilterExec: c@2 = 0", // even though data is sorted, we can use repartition here. Since // ordering is not used in subsequent stages anyway. - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", // SortExec doesn't benefit from input partitioning - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true); assert_optimized!(expected_csv, plan_csv, true); @@ -2567,35 +2576,35 @@ fn parallelization_ignores_limit() -> Result<()> { let expected_parquet = &[ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "GlobalLimitExec: skip=0, fetch=100", - "CoalescePartitionsExec", - "LocalLimitExec: fetch=100", - "FilterExec: c@2 = 0", + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " GlobalLimitExec: skip=0, fetch=100", + " CoalescePartitionsExec", + " LocalLimitExec: fetch=100", + " FilterExec: c@2 = 0", // repartition should happen prior to the filter to maximize parallelism - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "GlobalLimitExec: skip=0, fetch=100", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " GlobalLimitExec: skip=0, fetch=100", // Limit doesn't benefit from input partitioning - no parallelism - "LocalLimitExec: fetch=100", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " LocalLimitExec: fetch=100", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; let expected_csv = &[ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "GlobalLimitExec: skip=0, fetch=100", - "CoalescePartitionsExec", - "LocalLimitExec: fetch=100", - "FilterExec: c@2 = 0", + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " GlobalLimitExec: skip=0, fetch=100", + " CoalescePartitionsExec", + " LocalLimitExec: fetch=100", + " FilterExec: c@2 = 0", // repartition should happen prior to the filter to maximize parallelism - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "GlobalLimitExec: skip=0, fetch=100", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " GlobalLimitExec: skip=0, fetch=100", // Limit doesn't benefit from input partitioning - no parallelism - "LocalLimitExec: fetch=100", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + " LocalLimitExec: fetch=100", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true); assert_optimized!(expected_csv, plan_csv, true); @@ -2611,20 +2620,20 @@ fn parallelization_union_inputs() -> Result<()> { let expected_parquet = &[ "UnionExec", // Union doesn't benefit from input partitioning - no parallelism - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; let expected_csv = &[ "UnionExec", // Union doesn't benefit from input partitioning - no parallelism - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true); assert_optimized!(expected_csv, plan_csv, true); @@ -2678,15 +2687,15 @@ fn parallelization_sort_preserving_merge_with_union() -> Result<()> { // should not sort (as the data was already sorted) let expected_parquet = &[ "SortPreservingMergeExec: [c@2 ASC]", - "UnionExec", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + " UnionExec", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; let expected_csv = &[ "SortPreservingMergeExec: [c@2 ASC]", - "UnionExec", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", + " UnionExec", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true); assert_optimized!(expected_csv, plan_csv, true); @@ -2713,11 +2722,11 @@ fn parallelization_does_not_benefit() -> Result<()> { // no parallelization, because SortRequiredExec doesn't benefit from increased parallelism let expected_parquet = &[ "SortRequiredExec: [c@2 ASC]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; let expected_csv = &[ "SortRequiredExec: [c@2 ASC]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true); assert_optimized!(expected_csv, plan_csv, true); @@ -2757,7 +2766,7 @@ fn parallelization_ignores_transitively_with_projection_parquet() -> Result<()> // data should not be repartitioned / resorted let expected_parquet = &[ "ProjectionExec: expr=[a@0 as a2, c@2 as c2]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected_parquet, plan_parquet, true); @@ -2796,7 +2805,7 @@ fn parallelization_ignores_transitively_with_projection_csv() -> Result<()> { // data should not be repartitioned / resorted let expected_csv = &[ "ProjectionExec: expr=[a@0 as a2, c@2 as c2]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", ]; assert_optimized!(expected_csv, plan_csv, true); @@ -2819,8 +2828,8 @@ fn remove_redundant_roundrobins() -> Result<()> { let expected = &[ "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, physical_plan.clone(), true); assert_optimized!(expected, physical_plan, false); @@ -2842,9 +2851,9 @@ fn remove_unnecessary_spm_after_filter() -> Result<()> { // This is still satisfied since, after filter that column is constant. let expected = &[ "CoalescePartitionsExec", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=c@2 ASC", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + " FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=c@2 ASC", + " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; // last flag sets config.optimizer.PREFER_EXISTING_SORT assert_optimized!(expected, physical_plan.clone(), true, true); @@ -2865,9 +2874,9 @@ fn preserve_ordering_through_repartition() -> Result<()> { let expected = &[ "SortPreservingMergeExec: [d@3 ASC]", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=d@3 ASC", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet", + " FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=d@3 ASC", + " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet", ]; // last flag sets config.optimizer.PREFER_EXISTING_SORT assert_optimized!(expected, physical_plan.clone(), true, true); @@ -2888,20 +2897,20 @@ fn do_not_preserve_ordering_through_repartition() -> Result<()> { let expected = &[ "SortPreservingMergeExec: [a@0 ASC]", - "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + " FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; assert_optimized!(expected, physical_plan.clone(), true); let expected = &[ "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", + " CoalescePartitionsExec", + " FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; assert_optimized!(expected, physical_plan, false); @@ -2922,9 +2931,9 @@ fn no_need_for_sort_after_filter() -> Result<()> { // After CoalescePartitionsExec c is still constant. Hence c@2 ASC ordering is already satisfied. "CoalescePartitionsExec", // Since after this stage c is constant. c@2 ASC ordering is already satisfied. - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + " FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, physical_plan.clone(), true); assert_optimized!(expected, physical_plan, false); @@ -2949,21 +2958,21 @@ fn do_not_preserve_ordering_through_repartition2() -> Result<()> { let expected = &[ "SortPreservingMergeExec: [a@0 ASC]", - "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + " FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, physical_plan.clone(), true); let expected = &[ "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + " CoalescePartitionsExec", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + " FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, physical_plan, false); @@ -2982,8 +2991,8 @@ fn do_not_preserve_ordering_through_repartition3() -> Result<()> { let expected = &[ "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, physical_plan.clone(), true); assert_optimized!(expected, physical_plan, false); @@ -3004,8 +3013,8 @@ fn do_not_put_sort_when_input_is_invalid() -> Result<()> { // Ordering requirement of sort required exec is NOT satisfied // by existing ordering at the source. "SortRequiredExec: [a@0 ASC]", - "FilterExec: c@2 = 0", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " FilterExec: c@2 = 0", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_plan_txt!(expected, physical_plan); @@ -3013,9 +3022,9 @@ fn do_not_put_sort_when_input_is_invalid() -> Result<()> { "SortRequiredExec: [a@0 ASC]", // Since at the start of the rule ordering requirement is not satisfied // EnforceDistribution rule doesn't satisfy this requirement either. - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; let mut config = ConfigOptions::new(); @@ -3042,8 +3051,8 @@ fn put_sort_when_input_is_valid() -> Result<()> { // Ordering requirement of sort required exec is satisfied // by existing ordering at the source. "SortRequiredExec: [a@0 ASC]", - "FilterExec: c@2 = 0", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", + " FilterExec: c@2 = 0", + " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; assert_plan_txt!(expected, physical_plan); @@ -3051,8 +3060,8 @@ fn put_sort_when_input_is_valid() -> Result<()> { // Since at the start of the rule ordering requirement is satisfied // EnforceDistribution rule satisfy this requirement also. "SortRequiredExec: [a@0 ASC]", - "FilterExec: c@2 = 0", - "DataSourceExec: file_groups={10 groups: [[x:0..20], [y:0..20], [x:20..40], [y:20..40], [x:40..60], [y:40..60], [x:60..80], [y:60..80], [x:80..100], [y:80..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", + " FilterExec: c@2 = 0", + " DataSourceExec: file_groups={10 groups: [[x:0..20], [y:0..20], [x:20..40], [y:20..40], [x:40..60], [y:40..60], [x:60..80], [y:60..80], [x:80..100], [y:80..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; let mut config = ConfigOptions::new(); @@ -3078,8 +3087,8 @@ fn do_not_add_unnecessary_hash() -> Result<()> { let expected = &[ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; // Make sure target partition number is 1. In this case hash repartition is unnecessary assert_optimized!(expected, physical_plan.clone(), true, false, 1, false, 1024); @@ -3104,12 +3113,12 @@ fn do_not_add_unnecessary_hash2() -> Result<()> { "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", // Since hash requirements of this operator is satisfied. There shouldn't be // a hash repartition here - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + " AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + " RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", + " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + " RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2", + " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; // Make sure target partition number is larger than 2 (e.g partition number at the source). assert_optimized!(expected, physical_plan.clone(), true, false, 4, false, 1024); @@ -3153,9 +3162,9 @@ fn optimize_away_unnecessary_repartition2() -> Result<()> { let expected = &[ "FilterExec: c@2 = 0", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, physical_plan.clone(), true); assert_optimized!(expected, physical_plan, false); From c0d53adf8323b840d0adaa62ba868d6acd4ba886 Mon Sep 17 00:00:00 2001 From: Shruti Sharma <98698727+shruti2522@users.noreply.github.com> Date: Wed, 5 Mar 2025 16:17:48 +0530 Subject: [PATCH 05/33] add method SessionStateBuilder::new_with_default_features() (#14998) * add func SessionStateBuilder::new_with_defaults() * update name * add comment --- .../core/src/execution/session_state.rs | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index 2ec1ba07b9d1..d26d5a219e18 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -1023,6 +1023,9 @@ impl SessionStateBuilder { /// /// See [`Self::with_default_features`] to install the default set of functions, /// catalogs, etc. + /// + /// To create a `SessionStateBuilder` with default features such as functions, + /// please see [`Self::new_with_default_features`]. pub fn new() -> Self { Self { session_id: None, @@ -1144,6 +1147,22 @@ impl SessionStateBuilder { self } + /// Returns a new [`SessionStateBuilder`] with default features. + /// + /// This is equivalent to calling [`Self::new()`] followed by [`Self::with_default_features()`]. + /// + /// ``` + /// use datafusion::execution::session_state::SessionStateBuilder; + /// + /// // Create a new SessionState with default features + /// let session_state = SessionStateBuilder::new_with_default_features() + /// .with_session_id("my_session".to_string()) + /// .build(); + /// ``` + pub fn new_with_default_features() -> Self { + Self::new().with_default_features() + } + /// Set the session id. pub fn with_session_id(mut self, session_id: String) -> Self { self.session_id = Some(session_id); From 3dc212c9078c92f57ab7f58e75e1258130c772d0 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 5 Mar 2025 08:55:06 -0500 Subject: [PATCH 06/33] Implement `tree` explain for FilterExec (#15001) --- datafusion/physical-plan/src/display.rs | 35 ++++++++++++++++- datafusion/physical-plan/src/filter.rs | 3 +- .../sqllogictest/test_files/explain_tree.slt | 39 ++++++++++++++++--- 3 files changed, 68 insertions(+), 9 deletions(-) diff --git a/datafusion/physical-plan/src/display.rs b/datafusion/physical-plan/src/display.rs index 26e0a822f588..096b51540745 100644 --- a/datafusion/physical-plan/src/display.rs +++ b/datafusion/physical-plan/src/display.rs @@ -37,10 +37,41 @@ use super::{accept, ExecutionPlan, ExecutionPlanVisitor}; #[derive(Debug, Clone, Copy)] pub enum DisplayFormatType { /// Default, compact format. Example: `FilterExec: c12 < 10.0` + /// + /// This format is designed to provide a detailed textual description + /// of all rele Default, - /// Verbose, showing all available details + /// Verbose, showing all available details. + /// + /// This form is even more detailed than [`Self::Default`] Verbose, - /// TreeRender, display plan like a tree. + /// TreeRender, displayed in the `tree` explain type. + /// + /// This format is inspired by DuckDB's explain plans. The information + /// presented should be "user friendly", and contain only the most relevant + /// information for understanding a plan. It should NOT contain the same level + /// of detail information as the [`Self::Default`] format. + /// + /// In this mode, each line contains a key=value pair. + /// Everything before the first `=` is treated as the key, and everything after the + /// first `=` is treated as the value. + /// + /// For example, if the output of `TreeRender` is this: + /// ```text + /// partition_sizes=[1] + /// partitions=1 + /// ``` + /// + /// It is rendered in the center of a box in the following way: + /// + /// ```text + /// ┌───────────────────────────┐ + /// │ DataSourceExec │ + /// │ -------------------- │ + /// │ partition_sizes: [1] │ + /// │ partitions: 1 │ + /// └───────────────────────────┘ + /// ``` TreeRender, } diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 524da586f52e..ffcda1d888b0 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -330,8 +330,7 @@ impl DisplayAs for FilterExec { write!(f, "FilterExec: {}{}", self.predicate, display_projections) } DisplayFormatType::TreeRender => { - // TODO: collect info - write!(f, "") + write!(f, "predicate={}", self.predicate) } } } diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index 5f0785782008..18de2c6a617f 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -74,13 +74,16 @@ physical_plan 03)└─────────────┬─────────────┘ 04)┌─────────────┴─────────────┐ 05)│ FilterExec │ -06)└─────────────┬─────────────┘ -07)┌─────────────┴─────────────┐ -08)│ RepartitionExec │ +06)│ -------------------- │ +07)│ predicate: │ +08)│ string_col@1 != foo │ 09)└─────────────┬─────────────┘ 10)┌─────────────┴─────────────┐ -11)│ DataSourceExec │ -12)└───────────────────────────┘ +11)│ RepartitionExec │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ DataSourceExec │ +15)└───────────────────────────┘ # Aggregate query TT @@ -185,6 +188,32 @@ physical_plan 26)│ DataSourceExec ││ DataSourceExec │ 27)└───────────────────────────┘└───────────────────────────┘ +# Long Filter (demonstrate what happens with wrapping) +query TT +explain SELECT int_col FROM table1 +WHERE string_col != 'foo' AND string_col != 'bar' AND string_col != 'a really long string constant' +; +---- +logical_plan +01)Projection: table1.int_col +02)--Filter: table1.string_col != Utf8("foo") AND table1.string_col != Utf8("bar") AND table1.string_col != Utf8("a really long string constant") +03)----TableScan: table1 projection=[int_col, string_col], partial_filters=[table1.string_col != Utf8("foo"), table1.string_col != Utf8("bar"), table1.string_col != Utf8("a really long string constant")] +physical_plan +01)┌───────────────────────────┐ +02)│ CoalesceBatchesExec │ +03)└─────────────┬─────────────┘ +04)┌─────────────┴─────────────┐ +05)│ FilterExec │ +06)│ -------------------- │ +07)│ predicate: │ +08)│string_col@1 != foo AND ...│ +09)└─────────────┬─────────────┘ +10)┌─────────────┴─────────────┐ +11)│ RepartitionExec │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ DataSourceExec │ +15)└───────────────────────────┘ # cleanup From ff6731401a8aa6812e8d118cbc7d70d7e62407e9 Mon Sep 17 00:00:00 2001 From: Peter L Date: Thu, 6 Mar 2025 02:12:31 +1030 Subject: [PATCH 07/33] Unparser add `AtArrow` and `ArrowAt` conversion to BinaryOperator (#14968) --- datafusion/sql/src/unparser/expr.rs | 20 ++++++++++++++++++-- 1 file changed, 18 insertions(+), 2 deletions(-) diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index a48c077afd9b..bf6361312727 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -948,8 +948,8 @@ impl Unparser<'_> { Operator::BitwiseShiftRight => Ok(BinaryOperator::PGBitwiseShiftRight), Operator::BitwiseShiftLeft => Ok(BinaryOperator::PGBitwiseShiftLeft), Operator::StringConcat => Ok(BinaryOperator::StringConcat), - Operator::AtArrow => not_impl_err!("unsupported operation: {op:?}"), - Operator::ArrowAt => not_impl_err!("unsupported operation: {op:?}"), + Operator::AtArrow => Ok(BinaryOperator::AtArrow), + Operator::ArrowAt => Ok(BinaryOperator::ArrowAt), } } @@ -2113,6 +2113,22 @@ mod tests { ))), "[1, 2, 3]", ), + ( + Expr::BinaryExpr(BinaryExpr { + left: Box::new(col("a")), + op: Operator::ArrowAt, + right: Box::new(col("b")), + }), + "(a <@ b)", + ), + ( + Expr::BinaryExpr(BinaryExpr { + left: Box::new(col("a")), + op: Operator::AtArrow, + right: Box::new(col("b")), + }), + "(a @> b)", + ), ]; for (expr, expected) in tests { From dfb711ff0692733eaae8a08c99ab20e08962ce87 Mon Sep 17 00:00:00 2001 From: Ruihang Xia Date: Wed, 5 Mar 2025 09:14:07 -0800 Subject: [PATCH 08/33] feat: Add dependency checks to verify-release-candidate script (#15009) Signed-off-by: Ruihang Xia --- dev/release/verify-release-candidate.sh | 34 +++++++++++++++++++++++++ 1 file changed, 34 insertions(+) diff --git a/dev/release/verify-release-candidate.sh b/dev/release/verify-release-candidate.sh index a053569dcb24..9ecbe1bc1713 100755 --- a/dev/release/verify-release-candidate.sh +++ b/dev/release/verify-release-candidate.sh @@ -18,6 +18,37 @@ # under the License. # +# Check that required dependencies are installed +check_dependencies() { + local missing_deps=0 + local required_deps=("curl" "git" "gpg" "cc" "protoc") + + # Either shasum or sha256sum/sha512sum are required + local has_sha_tools=0 + + for dep in "${required_deps[@]}"; do + if ! command -v $dep &> /dev/null; then + echo "Error: $dep is not installed or not in PATH" + missing_deps=1 + fi + done + + # Check for either shasum or sha256sum/sha512sum + if command -v shasum &> /dev/null; then + has_sha_tools=1 + elif command -v sha256sum &> /dev/null && command -v sha512sum &> /dev/null; then + has_sha_tools=1 + else + echo "Error: Neither shasum nor sha256sum/sha512sum are installed or in PATH" + missing_deps=1 + fi + + if [ $missing_deps -ne 0 ]; then + echo "Please install missing dependencies and try again" + exit 1 + fi +} + case $# in 2) VERSION="$1" RC_NUMBER="$2" @@ -31,6 +62,9 @@ set -e set -x set -o pipefail +# Add the dependency check early in the script execution +check_dependencies + SOURCE_DIR="$(cd "$(dirname "${BASH_SOURCE[0]:-$0}")" && pwd)" ARROW_DIR="$(dirname $(dirname ${SOURCE_DIR}))" ARROW_DIST_URL='https://dist.apache.org/repos/dist/dev/datafusion' From d288b80203eba2906b830ea7bed9828e77eced1c Mon Sep 17 00:00:00 2001 From: Yongting You <2010youy01@gmail.com> Date: Thu, 6 Mar 2025 04:05:03 +0800 Subject: [PATCH 09/33] fix: External sort failing on an edge case (#15017) * fix external sort failure * clippy * review --- datafusion/core/tests/memory_limit/mod.rs | 25 +++++++++++++++++ datafusion/physical-plan/src/sorts/sort.rs | 32 ++++++++++++++-------- 2 files changed, 45 insertions(+), 12 deletions(-) diff --git a/datafusion/core/tests/memory_limit/mod.rs b/datafusion/core/tests/memory_limit/mod.rs index 2deb8fde2da6..8f690edc5426 100644 --- a/datafusion/core/tests/memory_limit/mod.rs +++ b/datafusion/core/tests/memory_limit/mod.rs @@ -468,6 +468,31 @@ async fn test_stringview_external_sort() { let _ = df.collect().await.expect("Query execution failed"); } +/// This test case is for a previously detected bug: +/// When `ExternalSorter` has read all input batches +/// - It has spilled many sorted runs to disk +/// - Its in-memory buffer for batches is almost full +/// The previous implementation will try to merge the spills and in-memory batches +/// together, without spilling the in-memory batches first, causing OOM. +#[tokio::test] +async fn test_in_mem_buffer_almost_full() { + let config = SessionConfig::new() + .with_sort_spill_reservation_bytes(3000000) + .with_target_partitions(1); + let runtime = RuntimeEnvBuilder::new() + .with_memory_pool(Arc::new(FairSpillPool::new(10 * 1024 * 1024))) + .build_arc() + .unwrap(); + + let ctx = SessionContext::new_with_config_rt(config, runtime); + + let query = "select * from generate_series(1,9000000) as t1(v1) order by v1;"; + let df = ctx.sql(query).await.unwrap(); + + // Check not fail + let _ = df.collect().await.unwrap(); +} + /// Run the query with the specified memory limit, /// and verifies the expected errors are returned #[derive(Clone, Debug)] diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index a6c5dbec74dc..55ba77096a7d 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -307,7 +307,7 @@ impl ExternalSorter { let size = get_reserved_byte_for_record_batch(&input); if self.reservation.try_grow(size).is_err() { - self.sort_or_spill_in_mem_batches().await?; + self.sort_or_spill_in_mem_batches(false).await?; // We've already freed more than half of reserved memory, // so we can grow the reservation again. There's nothing we can do // if this try_grow fails. @@ -332,7 +332,7 @@ impl ExternalSorter { /// /// 2. A combined streaming merge incorporating both in-memory /// batches and data from spill files on disk. - fn sort(&mut self) -> Result { + async fn sort(&mut self) -> Result { // Release the memory reserved for merge back to the pool so // there is some left when `in_mem_sort_stream` requests an // allocation. @@ -340,10 +340,12 @@ impl ExternalSorter { if self.spilled_before() { let mut streams = vec![]; + + // Sort `in_mem_batches` and spill it first. If there are many + // `in_mem_batches` and the memory limit is almost reached, merging + // them with the spilled files at the same time might cause OOM. if !self.in_mem_batches.is_empty() { - let in_mem_stream = - self.in_mem_sort_stream(self.metrics.baseline.intermediate())?; - streams.push(in_mem_stream); + self.sort_or_spill_in_mem_batches(true).await?; } for spill in self.spills.drain(..) { @@ -488,11 +490,17 @@ impl ExternalSorter { /// the memory usage has dropped by a factor of 2, then we don't have /// to spill. Otherwise, we spill to free up memory for inserting /// more batches. - /// /// The factor of 2 aims to avoid a degenerate case where the /// memory required for `fetch` is just under the memory available, - // causing repeated re-sorting of data - async fn sort_or_spill_in_mem_batches(&mut self) -> Result<()> { + /// causing repeated re-sorting of data + /// + /// # Arguments + /// + /// * `force_spill` - If true, the method will spill the in-memory batches + /// even if the memory usage has not dropped by a factor of 2. Otherwise it will + /// only spill when the memory usage has dropped by the pre-defined factor. + /// + async fn sort_or_spill_in_mem_batches(&mut self, force_spill: bool) -> Result<()> { // Release the memory reserved for merge back to the pool so // there is some left when `in_mem_sort_stream` requests an // allocation. At the end of this function, memory will be @@ -529,7 +537,7 @@ impl ExternalSorter { // Sorting may free up some memory especially when fetch is `Some`. If we have // not freed more than 50% of the memory, then we have to spill to free up more // memory for inserting more batches. - if self.reservation.size() > before / 2 { + if (self.reservation.size() > before / 2) || force_spill { // We have not freed more than 50% of the memory, so we have to spill to // free up more memory self.spill().await?; @@ -1114,7 +1122,7 @@ impl ExecutionPlan for SortExec { let batch = batch?; sorter.insert_batch(batch).await?; } - sorter.sort() + sorter.sort().await }) .try_flatten(), ))) @@ -1409,7 +1417,7 @@ mod tests { // bytes. We leave a little wiggle room for the actual numbers. assert!((3..=10).contains(&spill_count)); assert!((9000..=10000).contains(&spilled_rows)); - assert!((36000..=40000).contains(&spilled_bytes)); + assert!((38000..=42000).contains(&spilled_bytes)); let columns = result[0].columns(); @@ -1482,7 +1490,7 @@ mod tests { // `number_of_batches / (sort_spill_reservation_bytes / batch_size)` assert!((12..=18).contains(&spill_count)); assert!((15000..=20000).contains(&spilled_rows)); - assert!((700000..=900000).contains(&spilled_bytes)); + assert!((900000..=1000000).contains(&spilled_bytes)); // Verify that the result is sorted let concated_result = concat_batches(&schema, &result)?; From 66c2988f00a7e9db5cbf614c7518496188b3b6e1 Mon Sep 17 00:00:00 2001 From: kosiew Date: Thu, 6 Mar 2025 05:44:55 +0800 Subject: [PATCH 10/33] Fix: to_char Function Now Correctly Handles DATE Values in DataFusion (#14970) * revert _to_char_scalar to before #14908 * Amend formatted loop - handle null * revert _to_char_array to before #14908 * add slt test --- datafusion/functions/src/datetime/to_char.rs | 58 ++++--------------- .../sqllogictest/test_files/timestamps.slt | 20 +++++++ 2 files changed, 31 insertions(+), 47 deletions(-) diff --git a/datafusion/functions/src/datetime/to_char.rs b/datafusion/functions/src/datetime/to_char.rs index 034bbb705070..8b2e5ad87471 100644 --- a/datafusion/functions/src/datetime/to_char.rs +++ b/datafusion/functions/src/datetime/to_char.rs @@ -212,14 +212,6 @@ fn _to_char_scalar( let is_scalar_expression = matches!(&expression, ColumnarValue::Scalar(_)); let array = expression.into_array(1)?; - // fix https://github.com/apache/datafusion/issues/14884 - // If the input date/time is null, return a null Utf8 result. - if array.is_null(0) { - return Ok(match is_scalar_expression { - true => ColumnarValue::Scalar(ScalarValue::Utf8(None)), - false => ColumnarValue::Array(new_null_array(&Utf8, array.len())), - }); - } if format.is_none() { if is_scalar_expression { return Ok(ColumnarValue::Scalar(ScalarValue::Utf8(None))); @@ -234,15 +226,21 @@ fn _to_char_scalar( }; let formatter = ArrayFormatter::try_new(array.as_ref(), &format_options)?; - let formatted: Result, ArrowError> = (0..array.len()) - .map(|i| formatter.value(i).try_to_string()) + let formatted: Result>, ArrowError> = (0..array.len()) + .map(|i| { + if array.is_null(i) { + Ok(None) + } else { + formatter.value(i).try_to_string().map(Some) + } + }) .collect(); if let Ok(formatted) = formatted { if is_scalar_expression { - Ok(ColumnarValue::Scalar(ScalarValue::Utf8(Some( - formatted.first().unwrap().to_string(), - )))) + Ok(ColumnarValue::Scalar(ScalarValue::Utf8( + formatted.first().unwrap().clone(), + ))) } else { Ok(ColumnarValue::Array( Arc::new(StringArray::from(formatted)) as ArrayRef @@ -260,13 +258,6 @@ fn _to_char_array(args: &[ColumnarValue]) -> Result { let data_type = arrays[0].data_type(); for idx in 0..arrays[0].len() { - // fix https://github.com/apache/datafusion/issues/14884 - // If the date/time value is null, push None. - if arrays[0].is_null(idx) { - results.push(None); - continue; - } - let format = if format_array.is_null(idx) { None } else { @@ -678,31 +669,4 @@ mod tests { "Execution error: Format for `to_char` must be non-null Utf8, received Timestamp(Nanosecond, None)" ); } - - #[test] - fn test_to_char_input_none_array() { - let date_array = Arc::new(Date32Array::from(vec![Some(18506), None])) as ArrayRef; - let format_array = - StringArray::from(vec!["%Y-%m-%d".to_string(), "%Y-%m-%d".to_string()]); - let args = datafusion_expr::ScalarFunctionArgs { - args: vec![ - ColumnarValue::Array(date_array), - ColumnarValue::Array(Arc::new(format_array) as ArrayRef), - ], - number_rows: 2, - return_type: &DataType::Utf8, - }; - let result = ToCharFunc::new() - .invoke_with_args(args) - .expect("Expected no error"); - if let ColumnarValue::Array(result) = result { - let result = result.as_any().downcast_ref::().unwrap(); - assert_eq!(result.len(), 2); - // The first element is valid, second is null. - assert!(!result.is_null(0)); - assert!(result.is_null(1)); - } else { - panic!("Expected an array value"); - } - } } diff --git a/datafusion/sqllogictest/test_files/timestamps.slt b/datafusion/sqllogictest/test_files/timestamps.slt index 7eadb3c89dac..dcbcfbfa439d 100644 --- a/datafusion/sqllogictest/test_files/timestamps.slt +++ b/datafusion/sqllogictest/test_files/timestamps.slt @@ -2847,6 +2847,26 @@ SELECT to_char(null, '%d-%m-%Y'); ---- NULL +query T +SELECT to_char(date_column, '%Y-%m-%d') +FROM (VALUES + (DATE '2020-09-01'), + (NULL) +) AS t(date_column); +---- +2020-09-01 +NULL + +query T +SELECT to_char(date_column, '%Y-%m-%d') +FROM (VALUES + (NULL), + (DATE '2020-09-01') +) AS t(date_column); +---- +NULL +2020-09-01 + query T SELECT to_char(column1, column2) FROM From 9a4c9d598ca93ef66ba02067cc1497d53c28cddc Mon Sep 17 00:00:00 2001 From: Bryce Mecum Date: Wed, 5 Mar 2025 13:51:42 -0800 Subject: [PATCH 11/33] Switch Unspecified to Required (#15011) --- datafusion/substrait/src/logical_plan/producer.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index e4df9703b20c..fc24d5bb91f0 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -1039,7 +1039,7 @@ fn to_substrait_named_struct(schema: &DFSchemaRef) -> Result { .map(|f| to_substrait_type(f.data_type(), f.is_nullable())) .collect::>()?, type_variation_reference: DEFAULT_TYPE_VARIATION_REF, - nullability: r#type::Nullability::Unspecified as i32, + nullability: r#type::Nullability::Required as i32, }; Ok(NamedStruct { From 43ecd9b807877946706628633308f73a4645de1f Mon Sep 17 00:00:00 2001 From: Li-Lun Lin <70696274+alan910127@users.noreply.github.com> Date: Thu, 6 Mar 2025 08:06:02 +0800 Subject: [PATCH 12/33] fix: graceful NULL and type error handling in array functions (#14737) * feat: arbitrary typed argument in array function * fix: array_sort null handling * fix: array_resize signature * test: add array_sort sqllogictest for null and invalid types * fix: don't match error message * chore: use string instead of data type * refactor: use new_null_array * fix: pass null to array argument should return null * fix: handle null argument for array in replace and resize * fix: mismatched error message * fix: incorrect number of rows returned * test: update null tests * fix: treat NULLs as lists directly to prevent extra handling * fix: incorrect null pushing in array_sort --- datafusion/expr-common/src/signature.rs | 5 ++ .../expr/src/type_coercion/functions.rs | 9 ++- datafusion/functions-nested/src/extract.rs | 1 + datafusion/functions-nested/src/replace.rs | 7 +- datafusion/functions-nested/src/resize.rs | 32 +++++++-- datafusion/functions-nested/src/sort.rs | 51 +++++++++++--- datafusion/sqllogictest/test_files/array.slt | 69 ++++++++++++++++--- 7 files changed, 149 insertions(+), 25 deletions(-) diff --git a/datafusion/expr-common/src/signature.rs b/datafusion/expr-common/src/signature.rs index ba6fadbf7235..063417a254be 100644 --- a/datafusion/expr-common/src/signature.rs +++ b/datafusion/expr-common/src/signature.rs @@ -358,6 +358,8 @@ pub enum ArrayFunctionArgument { /// An argument of type List/LargeList/FixedSizeList. All Array arguments must be coercible /// to the same type. Array, + // A Utf8 argument. + String, } impl Display for ArrayFunctionArgument { @@ -372,6 +374,9 @@ impl Display for ArrayFunctionArgument { ArrayFunctionArgument::Array => { write!(f, "array") } + ArrayFunctionArgument::String => { + write!(f, "string") + } } } } diff --git a/datafusion/expr/src/type_coercion/functions.rs b/datafusion/expr/src/type_coercion/functions.rs index b471feca043f..0ec017bdc27f 100644 --- a/datafusion/expr/src/type_coercion/functions.rs +++ b/datafusion/expr/src/type_coercion/functions.rs @@ -19,7 +19,7 @@ use super::binary::{binary_numeric_coercion, comparison_coercion}; use crate::{AggregateUDF, ScalarUDF, Signature, TypeSignature, WindowUDF}; use arrow::{ compute::can_cast_types, - datatypes::{DataType, TimeUnit}, + datatypes::{DataType, Field, TimeUnit}, }; use datafusion_common::types::LogicalType; use datafusion_common::utils::{coerced_fixed_size_list_to_list, ListCoercion}; @@ -387,7 +387,7 @@ fn get_valid_types( new_base_type = coerce_array_types(function_name, current_type, &new_base_type)?; } - ArrayFunctionArgument::Index => {} + ArrayFunctionArgument::Index | ArrayFunctionArgument::String => {} } } let new_array_type = datafusion_common::utils::coerced_type_with_base_type_only( @@ -408,6 +408,7 @@ fn get_valid_types( let valid_type = match argument_type { ArrayFunctionArgument::Element => new_elem_type.clone(), ArrayFunctionArgument::Index => DataType::Int64, + ArrayFunctionArgument::String => DataType::Utf8, ArrayFunctionArgument::Array => { let Some(current_type) = array(current_type) else { return Ok(vec![vec![]]); @@ -435,6 +436,10 @@ fn get_valid_types( match array_type { DataType::List(_) | DataType::LargeList(_) => Some(array_type.clone()), DataType::FixedSizeList(field, _) => Some(DataType::List(Arc::clone(field))), + DataType::Null => Some(DataType::List(Arc::new(Field::new_list_field( + DataType::Int64, + true, + )))), _ => None, } } diff --git a/datafusion/functions-nested/src/extract.rs b/datafusion/functions-nested/src/extract.rs index 422b1b612850..0f50f62dd8d2 100644 --- a/datafusion/functions-nested/src/extract.rs +++ b/datafusion/functions-nested/src/extract.rs @@ -166,6 +166,7 @@ impl ScalarUDFImpl for ArrayElement { List(field) | LargeList(field) | FixedSizeList(field, _) => Ok(field.data_type().clone()), + DataType::Null => Ok(List(Arc::new(Field::new_list_field(DataType::Int64, true)))), _ => plan_err!( "ArrayElement can only accept List, LargeList or FixedSizeList as the first argument" ), diff --git a/datafusion/functions-nested/src/replace.rs b/datafusion/functions-nested/src/replace.rs index 71bfedb72d1c..3dbe672c5b02 100644 --- a/datafusion/functions-nested/src/replace.rs +++ b/datafusion/functions-nested/src/replace.rs @@ -18,8 +18,8 @@ //! [`ScalarUDFImpl`] definitions for array_replace, array_replace_n and array_replace_all functions. use arrow::array::{ - Array, ArrayRef, AsArray, Capacities, GenericListArray, MutableArrayData, - NullBufferBuilder, OffsetSizeTrait, + new_null_array, Array, ArrayRef, AsArray, Capacities, GenericListArray, + MutableArrayData, NullBufferBuilder, OffsetSizeTrait, }; use arrow::datatypes::{DataType, Field}; @@ -429,6 +429,7 @@ pub(crate) fn array_replace_inner(args: &[ArrayRef]) -> Result { let list_array = array.as_list::(); general_replace::(list_array, from, to, arr_n) } + DataType::Null => Ok(new_null_array(array.data_type(), 1)), array_type => exec_err!("array_replace does not support type '{array_type:?}'."), } } @@ -447,6 +448,7 @@ pub(crate) fn array_replace_n_inner(args: &[ArrayRef]) -> Result { let list_array = array.as_list::(); general_replace::(list_array, from, to, arr_n) } + DataType::Null => Ok(new_null_array(array.data_type(), 1)), array_type => { exec_err!("array_replace_n does not support type '{array_type:?}'.") } @@ -467,6 +469,7 @@ pub(crate) fn array_replace_all_inner(args: &[ArrayRef]) -> Result { let list_array = array.as_list::(); general_replace::(list_array, from, to, arr_n) } + DataType::Null => Ok(new_null_array(array.data_type(), 1)), array_type => { exec_err!("array_replace_all does not support type '{array_type:?}'.") } diff --git a/datafusion/functions-nested/src/resize.rs b/datafusion/functions-nested/src/resize.rs index 6c0b91a678e7..145d7e80043b 100644 --- a/datafusion/functions-nested/src/resize.rs +++ b/datafusion/functions-nested/src/resize.rs @@ -23,16 +23,18 @@ use arrow::array::{ MutableArrayData, NullBufferBuilder, OffsetSizeTrait, }; use arrow::buffer::OffsetBuffer; -use arrow::datatypes::ArrowNativeType; use arrow::datatypes::DataType; +use arrow::datatypes::{ArrowNativeType, Field}; use arrow::datatypes::{ DataType::{FixedSizeList, LargeList, List}, FieldRef, }; use datafusion_common::cast::{as_int64_array, as_large_list_array, as_list_array}; +use datafusion_common::utils::ListCoercion; use datafusion_common::{exec_err, internal_datafusion_err, Result, ScalarValue}; use datafusion_expr::{ - ColumnarValue, Documentation, ScalarUDFImpl, Signature, Volatility, + ArrayFunctionArgument, ArrayFunctionSignature, ColumnarValue, Documentation, + ScalarUDFImpl, Signature, TypeSignature, Volatility, }; use datafusion_macros::user_doc; use std::any::Any; @@ -83,7 +85,26 @@ impl Default for ArrayResize { impl ArrayResize { pub fn new() -> Self { Self { - signature: Signature::variadic_any(Volatility::Immutable), + signature: Signature::one_of( + vec![ + TypeSignature::ArraySignature(ArrayFunctionSignature::Array { + arguments: vec![ + ArrayFunctionArgument::Array, + ArrayFunctionArgument::Index, + ], + array_coercion: Some(ListCoercion::FixedSizedListToList), + }), + TypeSignature::ArraySignature(ArrayFunctionSignature::Array { + arguments: vec![ + ArrayFunctionArgument::Array, + ArrayFunctionArgument::Index, + ArrayFunctionArgument::Element, + ], + array_coercion: Some(ListCoercion::FixedSizedListToList), + }), + ], + Volatility::Immutable, + ), aliases: vec!["list_resize".to_string()], } } @@ -106,6 +127,9 @@ impl ScalarUDFImpl for ArrayResize { match &arg_types[0] { List(field) | FixedSizeList(field, _) => Ok(List(Arc::clone(field))), LargeList(field) => Ok(LargeList(Arc::clone(field))), + DataType::Null => { + Ok(List(Arc::new(Field::new_list_field(DataType::Int64, true)))) + } _ => exec_err!( "Not reachable, data_type should be List, LargeList or FixedSizeList" ), @@ -137,7 +161,7 @@ pub(crate) fn array_resize_inner(arg: &[ArrayRef]) -> Result { let array = &arg[0]; // Checks if entire array is null - if array.null_count() == array.len() { + if array.logical_null_count() == array.len() { let return_type = match array.data_type() { List(field) => List(Arc::clone(field)), LargeList(field) => LargeList(Arc::clone(field)), diff --git a/datafusion/functions-nested/src/sort.rs b/datafusion/functions-nested/src/sort.rs index 7dbf9f2b211e..1db245fe52fe 100644 --- a/datafusion/functions-nested/src/sort.rs +++ b/datafusion/functions-nested/src/sort.rs @@ -18,7 +18,7 @@ //! [`ScalarUDFImpl`] definitions for array_sort function. use crate::utils::make_scalar_function; -use arrow::array::{Array, ArrayRef, ListArray, NullBufferBuilder}; +use arrow::array::{new_null_array, Array, ArrayRef, ListArray, NullBufferBuilder}; use arrow::buffer::OffsetBuffer; use arrow::datatypes::DataType::{FixedSizeList, LargeList, List}; use arrow::datatypes::{DataType, Field}; @@ -26,7 +26,8 @@ use arrow::{compute, compute::SortOptions}; use datafusion_common::cast::{as_list_array, as_string_array}; use datafusion_common::{exec_err, Result}; use datafusion_expr::{ - ColumnarValue, Documentation, ScalarUDFImpl, Signature, Volatility, + ArrayFunctionArgument, ArrayFunctionSignature, ColumnarValue, Documentation, + ScalarUDFImpl, Signature, TypeSignature, Volatility, }; use datafusion_macros::user_doc; use std::any::Any; @@ -87,7 +88,30 @@ impl Default for ArraySort { impl ArraySort { pub fn new() -> Self { Self { - signature: Signature::variadic_any(Volatility::Immutable), + signature: Signature::one_of( + vec![ + TypeSignature::ArraySignature(ArrayFunctionSignature::Array { + arguments: vec![ArrayFunctionArgument::Array], + array_coercion: None, + }), + TypeSignature::ArraySignature(ArrayFunctionSignature::Array { + arguments: vec![ + ArrayFunctionArgument::Array, + ArrayFunctionArgument::String, + ], + array_coercion: None, + }), + TypeSignature::ArraySignature(ArrayFunctionSignature::Array { + arguments: vec![ + ArrayFunctionArgument::Array, + ArrayFunctionArgument::String, + ArrayFunctionArgument::String, + ], + array_coercion: None, + }), + ], + Volatility::Immutable, + ), aliases: vec!["list_sort".to_string()], } } @@ -115,6 +139,7 @@ impl ScalarUDFImpl for ArraySort { field.data_type().clone(), true, )))), + DataType::Null => Ok(DataType::Null), _ => exec_err!( "Not reachable, data_type should be List, LargeList or FixedSizeList" ), @@ -143,6 +168,10 @@ pub fn array_sort_inner(args: &[ArrayRef]) -> Result { return exec_err!("array_sort expects one to three arguments"); } + if args[1..].iter().any(|array| array.is_null(0)) { + return Ok(new_null_array(args[0].data_type(), args[0].len())); + } + let sort_option = match args.len() { 1 => None, 2 => { @@ -196,12 +225,16 @@ pub fn array_sort_inner(args: &[ArrayRef]) -> Result { .map(|a| a.as_ref()) .collect::>(); - let list_arr = ListArray::new( - Arc::new(Field::new_list_field(data_type, true)), - OffsetBuffer::from_lengths(array_lengths), - Arc::new(compute::concat(elements.as_slice())?), - buffer, - ); + let list_arr = if elements.is_empty() { + ListArray::new_null(Arc::new(Field::new_list_field(data_type, true)), row_count) + } else { + ListArray::new( + Arc::new(Field::new_list_field(data_type, true)), + OffsetBuffer::from_lengths(array_lengths), + Arc::new(compute::concat(elements.as_slice())?), + buffer, + ) + }; Ok(Arc::new(list_arr)) } diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 6b5b246aee51..c8f6a985bb22 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -1204,8 +1204,10 @@ select array_element([1, 2], NULL); ---- NULL -query error +query I select array_element(NULL, 2); +---- +NULL # array_element scalar function #1 (with positive index) query IT @@ -2265,6 +2267,52 @@ select array_sort([]); ---- [] +# test with null arguments +query ? +select array_sort(NULL); +---- +NULL + +query ? +select array_sort(column1, NULL) from arrays_values; +---- +NULL +NULL +NULL +NULL +NULL +NULL +NULL +NULL + +query ?? +select array_sort(column1, 'DESC', NULL), array_sort(column1, 'ASC', NULL) from arrays_values; +---- +NULL NULL +NULL NULL +NULL NULL +NULL NULL +NULL NULL +NULL NULL +NULL NULL +NULL NULL + +query ?? +select array_sort(column1, NULL, 'NULLS FIRST'), array_sort(column1, NULL, 'NULLS LAST') from arrays_values; +---- +NULL NULL +NULL NULL +NULL NULL +NULL NULL +NULL NULL +NULL NULL +NULL NULL +NULL NULL + +## test with argument of incorrect types +query error DataFusion error: Execution error: the second parameter of array_sort expects DESC or ASC +select array_sort([1, 3, null, 5, NULL, -5], 1), array_sort([1, 3, null, 5, NULL, -5], 'DESC', 1), array_sort([1, 3, null, 5, NULL, -5], 1, 1); + # test with empty row, the row that does not match the condition has row count 0 statement ok create table t1(a int, b int) as values (100, 1), (101, 2), (102, 3), (101, 2); @@ -2290,8 +2338,10 @@ select list_sort(make_array(1, 3, null, 5, NULL, -5)), list_sort(make_array(1, 3 # array_append with NULLs -query error +query ? select array_append(null, 1); +---- +[1] query error select array_append(null, [2, 3]); @@ -2539,8 +2589,10 @@ select array_append(column1, arrow_cast(make_array(1, 11, 111), 'FixedSizeList(3 # DuckDB: [4] # ClickHouse: Null # Since they dont have the same result, we just follow Postgres, return error -query error +query ? select array_prepend(4, NULL); +---- +[4] query ? select array_prepend(4, []); @@ -2575,11 +2627,10 @@ select array_prepend(null, [[1,2,3]]); query error select array_prepend([], []); -# DuckDB: [null] -# ClickHouse: [null] -# TODO: We may also return [null] -query error +query ? select array_prepend(null, null); +---- +[NULL] query ? select array_append([], null); @@ -5264,9 +5315,11 @@ NULL [3] [5] # array_ndims scalar function #1 #follow PostgreSQL -query error +query I select array_ndims(null); +---- +NULL query I select From 6d5f5cdf2889d1f2e5bcb708087b5e2a4b1da011 Mon Sep 17 00:00:00 2001 From: "xudong.w" Date: Thu, 6 Mar 2025 18:13:52 +0700 Subject: [PATCH 13/33] Adjust physical optimizer rule order (#15040) --- .../physical-optimizer/src/optimizer.rs | 8 +++--- .../sqllogictest/test_files/explain.slt | 6 ++--- datafusion/sqllogictest/test_files/window.slt | 26 +++++++++---------- 3 files changed, 19 insertions(+), 21 deletions(-) diff --git a/datafusion/physical-optimizer/src/optimizer.rs b/datafusion/physical-optimizer/src/optimizer.rs index 88f11f53491e..bab31150e250 100644 --- a/datafusion/physical-optimizer/src/optimizer.rs +++ b/datafusion/physical-optimizer/src/optimizer.rs @@ -121,6 +121,10 @@ impl PhysicalOptimizer { // into an `order by max(x) limit y`. In this case it will copy the limit value down // to the aggregation, allowing it to use only y number of accumulators. Arc::new(TopKAggregation::new()), + // The LimitPushdown rule tries to push limits down as far as possible, + // replacing operators with fetching variants, or adding limits + // past operators that support limit pushdown. + Arc::new(LimitPushdown::new()), // The ProjectionPushdown rule tries to push projections towards // the sources in the execution plan. As a result of this process, // a projection can disappear if it reaches the source providers, and @@ -128,10 +132,6 @@ impl PhysicalOptimizer { // are not present, the load of executors such as join or union will be // reduced by narrowing their input tables. Arc::new(ProjectionPushdown::new()), - // The LimitPushdown rule tries to push limits down as far as possible, - // replacing operators with fetching variants, or adding limits - // past operators that support limit pushdown. - Arc::new(LimitPushdown::new()), // The SanityCheckPlan rule checks whether the order and // distribution requirements of each node in the plan // is satisfied. It will also reject non-runnable query diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 16c61a1db6ee..d32ddd151243 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -247,8 +247,8 @@ physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true physical_plan after LimitAggregation SAME TEXT AS ABOVE -physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after LimitPushdown SAME TEXT AS ABOVE +physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after SanityCheckPlan SAME TEXT AS ABOVE physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true physical_plan_with_stats DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] @@ -323,8 +323,8 @@ physical_plan after OutputRequirements 01)GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] physical_plan after LimitAggregation SAME TEXT AS ABOVE -physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after LimitPushdown DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after SanityCheckPlan SAME TEXT AS ABOVE physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] physical_plan_with_schema DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] @@ -363,8 +363,8 @@ physical_plan after OutputRequirements 01)GlobalLimitExec: skip=0, fetch=10 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet physical_plan after LimitAggregation SAME TEXT AS ABOVE -physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after LimitPushdown DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet +physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after SanityCheckPlan SAME TEXT AS ABOVE physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet physical_plan_with_stats DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 61bb2f022709..1a9acc0f531a 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -2833,13 +2833,12 @@ logical_plan 06)----------Projection: CAST(annotated_data_infinite.inc_col AS Int64) AS __common_expr_1, annotated_data_infinite.ts, annotated_data_infinite.inc_col 07)------------TableScan: annotated_data_infinite projection=[ts, inc_col] physical_plan -01)ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, count1@2 as count1, count2@3 as count2] -02)--ProjectionExec: expr=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as sum1, sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum2, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as count1, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as count2, ts@1 as ts] -03)----GlobalLimitExec: skip=0, fetch=5 -04)------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -05)--------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }], mode=[Sorted] -06)----------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as __common_expr_1, ts@0 as ts, inc_col@1 as inc_col] -07)------------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] +01)ProjectionExec: expr=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as sum1, sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum2, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as count1, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as count2] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----BoundedWindowAggExec: wdw=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +04)------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }], mode=[Sorted] +05)--------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as __common_expr_1, ts@0 as ts, inc_col@1 as inc_col] +06)----------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] query IIII SELECT @@ -2879,13 +2878,12 @@ logical_plan 06)----------Projection: CAST(annotated_data_infinite.inc_col AS Int64) AS __common_expr_1, annotated_data_infinite.ts, annotated_data_infinite.inc_col 07)------------TableScan: annotated_data_infinite projection=[ts, inc_col] physical_plan -01)ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, count1@2 as count1, count2@3 as count2] -02)--ProjectionExec: expr=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as sum1, sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum2, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as count1, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as count2, ts@1 as ts] -03)----GlobalLimitExec: skip=0, fetch=5 -04)------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -05)--------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }], mode=[Sorted] -06)----------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as __common_expr_1, ts@0 as ts, inc_col@1 as inc_col] -07)------------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] +01)ProjectionExec: expr=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as sum1, sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum2, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as count1, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as count2] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----BoundedWindowAggExec: wdw=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +04)------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }], mode=[Sorted] +05)--------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as __common_expr_1, ts@0 as ts, inc_col@1 as inc_col] +06)----------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] query IIII From eb4ee6272c77a2724c75edc714a93a1dd3e2c13d Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Thu, 6 Mar 2025 19:35:20 +0800 Subject: [PATCH 14/33] Move `UnwrapCastInComparison` into `Simplifier` (#15012) * add unwrap in simplify expr * rm unwrap cast * return err * rename * fix * fmt * add unwrap_cast module to simplify expressions * tweak comment * Move tests * Rewrite to use simplifier schema * Update tests for simplify logic --------- Co-authored-by: Andrew Lamb --- datafusion/core/tests/sql/explain_analyze.rs | 7 +- datafusion/optimizer/src/lib.rs | 1 - datafusion/optimizer/src/optimizer.rs | 3 - .../simplify_expressions/expr_simplifier.rs | 92 +++- .../optimizer/src/simplify_expressions/mod.rs | 1 + .../unwrap_cast.rs} | 461 ++++++++---------- .../sqllogictest/test_files/explain.slt | 4 - 7 files changed, 291 insertions(+), 278 deletions(-) rename datafusion/optimizer/src/{unwrap_cast_in_comparison.rs => simplify_expressions/unwrap_cast.rs} (79%) diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 3bdc71a8eb99..e8ef34c2afe7 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -355,7 +355,8 @@ async fn csv_explain_verbose() { async fn csv_explain_inlist_verbose() { let ctx = SessionContext::new(); register_aggregate_csv_by_sql(&ctx).await; - let sql = "EXPLAIN VERBOSE SELECT c1 FROM aggregate_test_100 where c2 in (1,2,4)"; + // Inlist len <=3 case will be transformed to OR List so we test with len=4 + let sql = "EXPLAIN VERBOSE SELECT c1 FROM aggregate_test_100 where c2 in (1,2,4,5)"; let actual = execute(&ctx, sql).await; // Optimized by PreCastLitInComparisonExpressions rule @@ -368,12 +369,12 @@ async fn csv_explain_inlist_verbose() { // before optimization (Int64 literals) assert_contains!( &actual, - "aggregate_test_100.c2 IN ([Int64(1), Int64(2), Int64(4)])" + "aggregate_test_100.c2 IN ([Int64(1), Int64(2), Int64(4), Int64(5)])" ); // after optimization (casted to Int8) assert_contains!( &actual, - "aggregate_test_100.c2 IN ([Int8(1), Int8(2), Int8(4)])" + "aggregate_test_100.c2 IN ([Int8(1), Int8(2), Int8(4), Int8(5)])" ); } diff --git a/datafusion/optimizer/src/lib.rs b/datafusion/optimizer/src/lib.rs index 61ca9b31cd29..1280bf2f466e 100644 --- a/datafusion/optimizer/src/lib.rs +++ b/datafusion/optimizer/src/lib.rs @@ -60,7 +60,6 @@ pub mod replace_distinct_aggregate; pub mod scalar_subquery_to_join; pub mod simplify_expressions; pub mod single_distinct_to_groupby; -pub mod unwrap_cast_in_comparison; pub mod utils; #[cfg(test)] diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index 49bce3c1ce82..018ad8ace0e3 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -54,7 +54,6 @@ use crate::replace_distinct_aggregate::ReplaceDistinctWithAggregate; use crate::scalar_subquery_to_join::ScalarSubqueryToJoin; use crate::simplify_expressions::SimplifyExpressions; use crate::single_distinct_to_groupby::SingleDistinctToGroupBy; -use crate::unwrap_cast_in_comparison::UnwrapCastInComparison; use crate::utils::log_plan; /// `OptimizerRule`s transforms one [`LogicalPlan`] into another which @@ -243,7 +242,6 @@ impl Optimizer { let rules: Vec> = vec![ Arc::new(EliminateNestedUnion::new()), Arc::new(SimplifyExpressions::new()), - Arc::new(UnwrapCastInComparison::new()), Arc::new(ReplaceDistinctWithAggregate::new()), Arc::new(EliminateJoin::new()), Arc::new(DecorrelatePredicateSubquery::new()), @@ -266,7 +264,6 @@ impl Optimizer { // The previous optimizations added expressions and projections, // that might benefit from the following rules Arc::new(SimplifyExpressions::new()), - Arc::new(UnwrapCastInComparison::new()), Arc::new(CommonSubexprEliminate::new()), Arc::new(EliminateGroupByConstant::new()), Arc::new(OptimizeProjections::new()), diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index 840c108905a9..d5a1b84e6aff 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -32,7 +32,6 @@ use datafusion_common::{ tree_node::{Transformed, TransformedResult, TreeNode, TreeNodeRewriter}, }; use datafusion_common::{internal_err, DFSchema, DataFusionError, Result, ScalarValue}; -use datafusion_expr::simplify::ExprSimplifyResult; use datafusion_expr::{ and, lit, or, BinaryExpr, Case, ColumnarValue, Expr, Like, Operator, Volatility, WindowFunctionDefinition, @@ -42,14 +41,23 @@ use datafusion_expr::{ expr::{InList, InSubquery, WindowFunction}, utils::{iter_conjunction, iter_conjunction_owned}, }; +use datafusion_expr::{simplify::ExprSimplifyResult, Cast, TryCast}; use datafusion_physical_expr::{create_physical_expr, execution_props::ExecutionProps}; use super::inlist_simplifier::ShortenInListSimplifier; use super::utils::*; -use crate::analyzer::type_coercion::TypeCoercionRewriter; use crate::simplify_expressions::guarantees::GuaranteeRewriter; use crate::simplify_expressions::regex::simplify_regex_expr; +use crate::simplify_expressions::unwrap_cast::{ + is_cast_expr_and_support_unwrap_cast_in_comparison_for_binary, + is_cast_expr_and_support_unwrap_cast_in_comparison_for_inlist, + unwrap_cast_in_comparison_for_binary, +}; use crate::simplify_expressions::SimplifyInfo; +use crate::{ + analyzer::type_coercion::TypeCoercionRewriter, + simplify_expressions::unwrap_cast::try_cast_literal_to_type, +}; use indexmap::IndexSet; use regex::Regex; @@ -1742,6 +1750,86 @@ impl TreeNodeRewriter for Simplifier<'_, S> { } } + // ======================================= + // unwrap_cast_in_comparison + // ======================================= + // + // For case: + // try_cast/cast(expr as data_type) op literal + Expr::BinaryExpr(BinaryExpr { left, op, right }) + if is_cast_expr_and_support_unwrap_cast_in_comparison_for_binary( + info, &left, &right, + ) && op.supports_propagation() => + { + unwrap_cast_in_comparison_for_binary(info, left, right, op)? + } + // literal op try_cast/cast(expr as data_type) + // --> + // try_cast/cast(expr as data_type) op_swap literal + Expr::BinaryExpr(BinaryExpr { left, op, right }) + if is_cast_expr_and_support_unwrap_cast_in_comparison_for_binary( + info, &right, &left, + ) && op.supports_propagation() + && op.swap().is_some() => + { + unwrap_cast_in_comparison_for_binary( + info, + right, + left, + op.swap().unwrap(), + )? + } + // For case: + // try_cast/cast(expr as left_type) in (expr1,expr2,expr3) + Expr::InList(InList { + expr: mut left, + list, + negated, + }) if is_cast_expr_and_support_unwrap_cast_in_comparison_for_inlist( + info, &left, &list, + ) => + { + let (Expr::TryCast(TryCast { + expr: left_expr, .. + }) + | Expr::Cast(Cast { + expr: left_expr, .. + })) = left.as_mut() + else { + return internal_err!("Expect cast expr, but got {:?}", left)?; + }; + + let expr_type = info.get_data_type(left_expr)?; + let right_exprs = list + .into_iter() + .map(|right| { + match right { + Expr::Literal(right_lit_value) => { + // if the right_lit_value can be casted to the type of internal_left_expr + // we need to unwrap the cast for cast/try_cast expr, and add cast to the literal + let Some(value) = try_cast_literal_to_type(&right_lit_value, &expr_type) else { + internal_err!( + "Can't cast the list expr {:?} to type {:?}", + right_lit_value, &expr_type + )? + }; + Ok(lit(value)) + } + other_expr => internal_err!( + "Only support literal expr to optimize, but the expr is {:?}", + &other_expr + ), + } + }) + .collect::>>()?; + + Transformed::yes(Expr::InList(InList { + expr: std::mem::take(left_expr), + list: right_exprs, + negated, + })) + } + // no additional rewrites possible expr => Transformed::no(expr), }) diff --git a/datafusion/optimizer/src/simplify_expressions/mod.rs b/datafusion/optimizer/src/simplify_expressions/mod.rs index 46c066c11c0f..5fbee02e3909 100644 --- a/datafusion/optimizer/src/simplify_expressions/mod.rs +++ b/datafusion/optimizer/src/simplify_expressions/mod.rs @@ -23,6 +23,7 @@ mod guarantees; mod inlist_simplifier; mod regex; pub mod simplify_exprs; +mod unwrap_cast; mod utils; // backwards compatibility diff --git a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs b/datafusion/optimizer/src/simplify_expressions/unwrap_cast.rs similarity index 79% rename from datafusion/optimizer/src/unwrap_cast_in_comparison.rs rename to datafusion/optimizer/src/simplify_expressions/unwrap_cast.rs index e2b8a966cb92..7670bdf98bb4 100644 --- a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs +++ b/datafusion/optimizer/src/simplify_expressions/unwrap_cast.rs @@ -15,274 +15,176 @@ // specific language governing permissions and limitations // under the License. -//! [`UnwrapCastInComparison`] rewrites `CAST(col) = lit` to `col = CAST(lit)` +//! Unwrap casts in binary comparisons +//! +//! The functions in this module attempt to remove casts from +//! comparisons to literals ([`ScalarValue`]s) by applying the casts +//! to the literals if possible. It is inspired by the optimizer rule +//! `UnwrapCastInBinaryComparison` of Spark. +//! +//! Removing casts often improves performance because: +//! 1. The cast is done once (to the literal) rather than to every value +//! 2. Can enable other optimizations such as predicate pushdown that +//! don't support casting +//! +//! The rule is applied to expressions of the following forms: +//! +//! 1. `cast(left_expr as data_type) comparison_op literal_expr` +//! 2. `literal_expr comparison_op cast(left_expr as data_type)` +//! 3. `cast(literal_expr) IN (expr1, expr2, ...)` +//! 4. `literal_expr IN (cast(expr1) , cast(expr2), ...)` +//! +//! If the expression matches one of the forms above, the rule will +//! ensure the value of `literal` is in range(min, max) of the +//! expr's data_type, and if the scalar is within range, the literal +//! will be casted to the data type of expr on the other side, and the +//! cast will be removed from the other side. +//! +//! # Example +//! +//! If the DataType of c1 is INT32. Given the filter +//! +//! ```text +//! cast(c1 as INT64) > INT64(10)` +//! ``` +//! +//! This rule will remove the cast and rewrite the expression to: +//! +//! ```text +//! c1 > INT32(10) +//! ``` +//! use std::cmp::Ordering; -use std::mem; -use std::sync::Arc; -use crate::optimizer::ApplyOrder; -use crate::{OptimizerConfig, OptimizerRule}; - -use crate::utils::NamePreserver; use arrow::datatypes::{ DataType, TimeUnit, MAX_DECIMAL128_FOR_EACH_PRECISION, MIN_DECIMAL128_FOR_EACH_PRECISION, }; use arrow::temporal_conversions::{MICROSECONDS, MILLISECONDS, NANOSECONDS}; -use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRewriter}; -use datafusion_common::{internal_err, DFSchema, DFSchemaRef, Result, ScalarValue}; -use datafusion_expr::expr::{BinaryExpr, Cast, InList, TryCast}; -use datafusion_expr::utils::merge_schema; -use datafusion_expr::{lit, Expr, ExprSchemable, LogicalPlan}; - -/// [`UnwrapCastInComparison`] attempts to remove casts from -/// comparisons to literals ([`ScalarValue`]s) by applying the casts -/// to the literals if possible. It is inspired by the optimizer rule -/// `UnwrapCastInBinaryComparison` of Spark. -/// -/// Removing casts often improves performance because: -/// 1. The cast is done once (to the literal) rather than to every value -/// 2. Can enable other optimizations such as predicate pushdown that -/// don't support casting -/// -/// The rule is applied to expressions of the following forms: -/// -/// 1. `cast(left_expr as data_type) comparison_op literal_expr` -/// 2. `literal_expr comparison_op cast(left_expr as data_type)` -/// 3. `cast(literal_expr) IN (expr1, expr2, ...)` -/// 4. `literal_expr IN (cast(expr1) , cast(expr2), ...)` -/// -/// If the expression matches one of the forms above, the rule will -/// ensure the value of `literal` is in range(min, max) of the -/// expr's data_type, and if the scalar is within range, the literal -/// will be casted to the data type of expr on the other side, and the -/// cast will be removed from the other side. -/// -/// # Example -/// -/// If the DataType of c1 is INT32. Given the filter -/// -/// ```text -/// Filter: cast(c1 as INT64) > INT64(10)` -/// ``` -/// -/// This rule will remove the cast and rewrite the expression to: -/// -/// ```text -/// Filter: c1 > INT32(10) -/// ``` -/// -#[derive(Default, Debug)] -pub struct UnwrapCastInComparison {} - -impl UnwrapCastInComparison { - pub fn new() -> Self { - Self::default() +use datafusion_common::{internal_err, tree_node::Transformed}; +use datafusion_common::{Result, ScalarValue}; +use datafusion_expr::{lit, BinaryExpr}; +use datafusion_expr::{simplify::SimplifyInfo, Cast, Expr, Operator, TryCast}; + +pub(super) fn unwrap_cast_in_comparison_for_binary( + info: &S, + cast_expr: Box, + literal: Box, + op: Operator, +) -> Result> { + match (*cast_expr, *literal) { + ( + Expr::TryCast(TryCast { expr, .. }) | Expr::Cast(Cast { expr, .. }), + Expr::Literal(lit_value), + ) => { + let Ok(expr_type) = info.get_data_type(&expr) else { + return internal_err!("Can't get the data type of the expr {:?}", &expr); + }; + // if the lit_value can be casted to the type of internal_left_expr + // we need to unwrap the cast for cast/try_cast expr, and add cast to the literal + let Some(value) = try_cast_literal_to_type(&lit_value, &expr_type) else { + return internal_err!( + "Can't cast the literal expr {:?} to type {:?}", + &lit_value, + &expr_type + ); + }; + Ok(Transformed::yes(Expr::BinaryExpr(BinaryExpr { + left: expr, + op, + right: Box::new(lit(value)), + }))) + } + _ => internal_err!("Expect cast expr and literal"), } } -impl OptimizerRule for UnwrapCastInComparison { - fn name(&self) -> &str { - "unwrap_cast_in_comparison" - } - - fn apply_order(&self) -> Option { - Some(ApplyOrder::BottomUp) - } +pub(super) fn is_cast_expr_and_support_unwrap_cast_in_comparison_for_binary< + S: SimplifyInfo, +>( + info: &S, + expr: &Expr, + literal: &Expr, +) -> bool { + match (expr, literal) { + ( + Expr::TryCast(TryCast { + expr: left_expr, .. + }) + | Expr::Cast(Cast { + expr: left_expr, .. + }), + Expr::Literal(lit_val), + ) => { + let Ok(expr_type) = info.get_data_type(left_expr) else { + return false; + }; - fn supports_rewrite(&self) -> bool { - true - } + let Ok(lit_type) = info.get_data_type(literal) else { + return false; + }; - fn rewrite( - &self, - plan: LogicalPlan, - _config: &dyn OptimizerConfig, - ) -> Result> { - let mut schema = merge_schema(&plan.inputs()); - - if let LogicalPlan::TableScan(ts) = &plan { - let source_schema = DFSchema::try_from_qualified_schema( - ts.table_name.clone(), - &ts.source.schema(), - )?; - schema.merge(&source_schema); + try_cast_literal_to_type(lit_val, &expr_type).is_some() + && is_supported_type(&expr_type) + && is_supported_type(&lit_type) } + _ => false, + } +} - schema.merge(plan.schema()); +pub(super) fn is_cast_expr_and_support_unwrap_cast_in_comparison_for_inlist< + S: SimplifyInfo, +>( + info: &S, + expr: &Expr, + list: &[Expr], +) -> bool { + let (Expr::TryCast(TryCast { + expr: left_expr, .. + }) + | Expr::Cast(Cast { + expr: left_expr, .. + })) = expr + else { + return false; + }; - let mut expr_rewriter = UnwrapCastExprRewriter { - schema: Arc::new(schema), - }; + let Ok(expr_type) = info.get_data_type(left_expr) else { + return false; + }; - let name_preserver = NamePreserver::new(&plan); - plan.map_expressions(|expr| { - let original_name = name_preserver.save(&expr); - expr.rewrite(&mut expr_rewriter) - .map(|transformed| transformed.update_data(|e| original_name.restore(e))) - }) + if !is_supported_type(&expr_type) { + return false; } -} -struct UnwrapCastExprRewriter { - schema: DFSchemaRef, -} + for right in list { + let Ok(right_type) = info.get_data_type(right) else { + return false; + }; -impl TreeNodeRewriter for UnwrapCastExprRewriter { - type Node = Expr; - - fn f_up(&mut self, mut expr: Expr) -> Result> { - match &mut expr { - // For case: - // try_cast/cast(expr as data_type) op literal - // literal op try_cast/cast(expr as data_type) - Expr::BinaryExpr(BinaryExpr { left, op, right }) - if { - let Ok(left_type) = left.get_type(&self.schema) else { - return Ok(Transformed::no(expr)); - }; - let Ok(right_type) = right.get_type(&self.schema) else { - return Ok(Transformed::no(expr)); - }; - is_supported_type(&left_type) - && is_supported_type(&right_type) - && op.supports_propagation() - } => - { - match (left.as_mut(), right.as_mut()) { - ( - Expr::Literal(left_lit_value), - Expr::TryCast(TryCast { - expr: right_expr, .. - }) - | Expr::Cast(Cast { - expr: right_expr, .. - }), - ) => { - // if the left_lit_value can be cast to the type of expr - // we need to unwrap the cast for cast/try_cast expr, and add cast to the literal - let Ok(expr_type) = right_expr.get_type(&self.schema) else { - return Ok(Transformed::no(expr)); - }; - match expr_type { - // https://github.com/apache/datafusion/issues/12180 - DataType::Utf8View => Ok(Transformed::no(expr)), - _ => { - let Some(value) = - try_cast_literal_to_type(left_lit_value, &expr_type) - else { - return Ok(Transformed::no(expr)); - }; - **left = lit(value); - // unwrap the cast/try_cast for the right expr - **right = mem::take(right_expr); - Ok(Transformed::yes(expr)) - } - } - } - ( - Expr::TryCast(TryCast { - expr: left_expr, .. - }) - | Expr::Cast(Cast { - expr: left_expr, .. - }), - Expr::Literal(right_lit_value), - ) => { - // if the right_lit_value can be cast to the type of expr - // we need to unwrap the cast for cast/try_cast expr, and add cast to the literal - let Ok(expr_type) = left_expr.get_type(&self.schema) else { - return Ok(Transformed::no(expr)); - }; - match expr_type { - // https://github.com/apache/datafusion/issues/12180 - DataType::Utf8View => Ok(Transformed::no(expr)), - _ => { - let Some(value) = - try_cast_literal_to_type(right_lit_value, &expr_type) - else { - return Ok(Transformed::no(expr)); - }; - // unwrap the cast/try_cast for the left expr - **left = mem::take(left_expr); - **right = lit(value); - Ok(Transformed::yes(expr)) - } - } - } - _ => Ok(Transformed::no(expr)), - } - } - // For case: - // try_cast/cast(expr as left_type) in (expr1,expr2,expr3) - Expr::InList(InList { - expr: left, list, .. - }) => { - let (Expr::TryCast(TryCast { - expr: left_expr, .. - }) - | Expr::Cast(Cast { - expr: left_expr, .. - })) = left.as_mut() - else { - return Ok(Transformed::no(expr)); - }; - let Ok(expr_type) = left_expr.get_type(&self.schema) else { - return Ok(Transformed::no(expr)); - }; - if !is_supported_type(&expr_type) { - return Ok(Transformed::no(expr)); - } - let Ok(right_exprs) = list - .iter() - .map(|right| { - let right_type = right.get_type(&self.schema)?; - if !is_supported_type(&right_type) { - internal_err!( - "The type of list expr {} is not supported", - &right_type - )?; - } - match right { - Expr::Literal(right_lit_value) => { - // if the right_lit_value can be casted to the type of internal_left_expr - // we need to unwrap the cast for cast/try_cast expr, and add cast to the literal - let Some(value) = try_cast_literal_to_type(right_lit_value, &expr_type) else { - internal_err!( - "Can't cast the list expr {:?} to type {:?}", - right_lit_value, &expr_type - )? - }; - Ok(lit(value)) - } - other_expr => internal_err!( - "Only support literal expr to optimize, but the expr is {:?}", - &other_expr - ), - } - }) - .collect::>>() else { - return Ok(Transformed::no(expr)) - }; - **left = mem::take(left_expr); - *list = right_exprs; - Ok(Transformed::yes(expr)) - } - // TODO: handle other expr type and dfs visit them - _ => Ok(Transformed::no(expr)), + if !is_supported_type(&right_type) { + return false; + } + + match right { + Expr::Literal(lit_val) + if try_cast_literal_to_type(lit_val, &expr_type).is_some() => {} + _ => return false, } } + + true } -/// Returns true if [UnwrapCastExprRewriter] supports this data type +/// Returns true if unwrap_cast_in_comparison supports this data type fn is_supported_type(data_type: &DataType) -> bool { is_supported_numeric_type(data_type) || is_supported_string_type(data_type) || is_supported_dictionary_type(data_type) } -/// Returns true if [[UnwrapCastExprRewriter]] support this numeric type +/// Returns true if unwrap_cast_in_comparison support this numeric type fn is_supported_numeric_type(data_type: &DataType) -> bool { matches!( data_type, @@ -299,7 +201,7 @@ fn is_supported_numeric_type(data_type: &DataType) -> bool { ) } -/// Returns true if [UnwrapCastExprRewriter] supports casting this value as a string +/// Returns true if unwrap_cast_in_comparison supports casting this value as a string fn is_supported_string_type(data_type: &DataType) -> bool { matches!( data_type, @@ -307,14 +209,14 @@ fn is_supported_string_type(data_type: &DataType) -> bool { ) } -/// Returns true if [UnwrapCastExprRewriter] supports casting this value as a dictionary +/// Returns true if unwrap_cast_in_comparison supports casting this value as a dictionary fn is_supported_dictionary_type(data_type: &DataType) -> bool { matches!(data_type, DataType::Dictionary(_, inner) if is_supported_type(inner)) } /// Convert a literal value from one data type to another -fn try_cast_literal_to_type( +pub(super) fn try_cast_literal_to_type( lit_value: &ScalarValue, target_type: &DataType, ) -> Option { @@ -540,13 +442,16 @@ fn cast_between_timestamp(from: &DataType, to: &DataType, value: i128) -> Option #[cfg(test)] mod tests { - use std::collections::HashMap; - use super::*; + use std::collections::HashMap; + use std::sync::Arc; + use crate::simplify_expressions::ExprSimplifier; use arrow::compute::{cast_with_options, CastOptions}; use arrow::datatypes::Field; - use datafusion_common::tree_node::TransformedResult; + use datafusion_common::{DFSchema, DFSchemaRef}; + use datafusion_expr::execution_props::ExecutionProps; + use datafusion_expr::simplify::SimplifyContext; use datafusion_expr::{cast, col, in_list, try_cast}; #[test] @@ -587,9 +492,9 @@ mod tests { let expected = col("c1").lt(null_i32()); assert_eq!(optimize_test(c1_lt_lit_null, &schema), expected); - // cast(INT8(NULL), INT32) < INT32(12) => INT8(NULL) < INT8(12) + // cast(INT8(NULL), INT32) < INT32(12) => INT8(NULL) < INT8(12) => BOOL(NULL) let lit_lt_lit = cast(null_i8(), DataType::Int32).lt(lit(12i32)); - let expected = null_i8().lt(lit(12i8)); + let expected = null_bool(); assert_eq!(optimize_test(lit_lt_lit, &schema), expected); } @@ -623,7 +528,7 @@ mod tests { // Verify reversed argument order // arrow_cast('value', 'Dictionary') = cast(str1 as Dictionary) => Utf8('value1') = str1 let expr_input = lit(dict.clone()).eq(cast(col("str1"), dict.data_type())); - let expected = lit("value").eq(col("str1")); + let expected = col("str1").eq(lit("value")); assert_eq!(optimize_test(expr_input, &schema), expected); } @@ -740,15 +645,27 @@ mod tests { #[test] fn test_unwrap_list_cast_comparison() { let schema = expr_test_schema(); - // INT32(C1) IN (INT32(12),INT64(24)) -> INT32(C1) IN (INT32(12),INT32(24)) - let expr_lt = - cast(col("c1"), DataType::Int64).in_list(vec![lit(12i64), lit(24i64)], false); - let expected = col("c1").in_list(vec![lit(12i32), lit(24i32)], false); + // INT32(C1) IN (INT32(12),INT64(23),INT64(34),INT64(56),INT64(78)) -> + // INT32(C1) IN (INT32(12),INT32(23),INT32(34),INT32(56),INT32(78)) + let expr_lt = cast(col("c1"), DataType::Int64).in_list( + vec![lit(12i64), lit(23i64), lit(34i64), lit(56i64), lit(78i64)], + false, + ); + let expected = col("c1").in_list( + vec![lit(12i32), lit(23i32), lit(34i32), lit(56i32), lit(78i32)], + false, + ); assert_eq!(optimize_test(expr_lt, &schema), expected); - // INT32(C2) IN (INT64(NULL),INT64(24)) -> INT32(C1) IN (INT32(12),INT32(24)) - let expr_lt = - cast(col("c2"), DataType::Int32).in_list(vec![null_i32(), lit(14i32)], false); - let expected = col("c2").in_list(vec![null_i64(), lit(14i64)], false); + // INT32(C2) IN (INT64(NULL),INT64(24),INT64(34),INT64(56),INT64(78)) -> + // INT32(C2) IN (INT32(NULL),INT32(24),INT32(34),INT32(56),INT32(78)) + let expr_lt = cast(col("c2"), DataType::Int32).in_list( + vec![null_i32(), lit(24i32), lit(34i64), lit(56i64), lit(78i64)], + false, + ); + let expected = col("c2").in_list( + vec![null_i64(), lit(24i64), lit(34i64), lit(56i64), lit(78i64)], + false, + ); assert_eq!(optimize_test(expr_lt, &schema), expected); @@ -774,10 +691,14 @@ mod tests { ); assert_eq!(optimize_test(expr_lt, &schema), expected); - // cast(INT32(12), INT64) IN (.....) - let expr_lt = cast(lit(12i32), DataType::Int64) - .in_list(vec![lit(13i64), lit(12i64)], false); - let expected = lit(12i32).in_list(vec![lit(13i32), lit(12i32)], false); + // cast(INT32(12), INT64) IN (.....) => + // INT64(12) IN (INT64(12),INT64(13),INT64(14),INT64(15),INT64(16)) + // => true + let expr_lt = cast(lit(12i32), DataType::Int64).in_list( + vec![lit(12i64), lit(13i64), lit(14i64), lit(15i64), lit(16i64)], + false, + ); + let expected = lit(true); assert_eq!(optimize_test(expr_lt, &schema), expected); } @@ -815,8 +736,12 @@ mod tests { assert_eq!(optimize_test(expr_input.clone(), &schema), expr_input); // inlist for unsupported data type - let expr_input = - in_list(cast(col("c6"), DataType::Float64), vec![lit(0f64)], false); + let expr_input = in_list( + cast(col("c6"), DataType::Float64), + // need more literals to avoid rewriting to binary expr + vec![lit(0f64), lit(1f64), lit(2f64), lit(3f64), lit(4f64)], + false, + ); assert_eq!(optimize_test(expr_input.clone(), &schema), expr_input); } @@ -833,10 +758,12 @@ mod tests { } fn optimize_test(expr: Expr, schema: &DFSchemaRef) -> Expr { - let mut expr_rewriter = UnwrapCastExprRewriter { - schema: Arc::clone(schema), - }; - expr.rewrite(&mut expr_rewriter).data().unwrap() + let props = ExecutionProps::new(); + let simplifier = ExprSimplifier::new( + SimplifyContext::new(&props).with_schema(Arc::clone(schema)), + ); + + simplifier.simplify(expr).unwrap() } fn expr_test_schema() -> DFSchemaRef { @@ -862,6 +789,10 @@ mod tests { ) } + fn null_bool() -> Expr { + lit(ScalarValue::Boolean(None)) + } + fn null_i8() -> Expr { lit(ScalarValue::Int8(None)) } diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index d32ddd151243..cab7308f6ff8 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -181,7 +181,6 @@ logical_plan after type_coercion SAME TEXT AS ABOVE analyzed_logical_plan SAME TEXT AS ABOVE logical_plan after eliminate_nested_union SAME TEXT AS ABOVE logical_plan after simplify_expressions SAME TEXT AS ABOVE -logical_plan after unwrap_cast_in_comparison SAME TEXT AS ABOVE logical_plan after replace_distinct_aggregate SAME TEXT AS ABOVE logical_plan after eliminate_join SAME TEXT AS ABOVE logical_plan after decorrelate_predicate_subquery SAME TEXT AS ABOVE @@ -200,13 +199,11 @@ logical_plan after push_down_limit SAME TEXT AS ABOVE logical_plan after push_down_filter SAME TEXT AS ABOVE logical_plan after single_distinct_aggregation_to_group_by SAME TEXT AS ABOVE logical_plan after simplify_expressions SAME TEXT AS ABOVE -logical_plan after unwrap_cast_in_comparison SAME TEXT AS ABOVE logical_plan after common_sub_expression_eliminate SAME TEXT AS ABOVE logical_plan after eliminate_group_by_constant SAME TEXT AS ABOVE logical_plan after optimize_projections TableScan: simple_explain_test projection=[a, b, c] logical_plan after eliminate_nested_union SAME TEXT AS ABOVE logical_plan after simplify_expressions SAME TEXT AS ABOVE -logical_plan after unwrap_cast_in_comparison SAME TEXT AS ABOVE logical_plan after replace_distinct_aggregate SAME TEXT AS ABOVE logical_plan after eliminate_join SAME TEXT AS ABOVE logical_plan after decorrelate_predicate_subquery SAME TEXT AS ABOVE @@ -225,7 +222,6 @@ logical_plan after push_down_limit SAME TEXT AS ABOVE logical_plan after push_down_filter SAME TEXT AS ABOVE logical_plan after single_distinct_aggregation_to_group_by SAME TEXT AS ABOVE logical_plan after simplify_expressions SAME TEXT AS ABOVE -logical_plan after unwrap_cast_in_comparison SAME TEXT AS ABOVE logical_plan after common_sub_expression_eliminate SAME TEXT AS ABOVE logical_plan after eliminate_group_by_constant SAME TEXT AS ABOVE logical_plan after optimize_projections SAME TEXT AS ABOVE From 8e72c48d4e007784fc9a5819036b2738f6a2e256 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 6 Mar 2025 06:41:21 -0500 Subject: [PATCH 15/33] chore(deps): bump aws-config from 1.5.17 to 1.5.18 (#15041) Bumps [aws-config](https://github.com/smithy-lang/smithy-rs) from 1.5.17 to 1.5.18. - [Release notes](https://github.com/smithy-lang/smithy-rs/releases) - [Changelog](https://github.com/smithy-lang/smithy-rs/blob/main/CHANGELOG.md) - [Commits](https://github.com/smithy-lang/smithy-rs/commits) --- updated-dependencies: - dependency-name: aws-config dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 50 +++++++++++++++++++++++++++------------ datafusion-cli/Cargo.toml | 2 +- 2 files changed, 36 insertions(+), 16 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 069462b942e5..6e2a772a1370 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -607,9 +607,9 @@ checksum = "ace50bade8e6234aa140d9a2f552bbee1db4d353f69b8217bc503490fc1a9f26" [[package]] name = "aws-config" -version = "1.5.17" +version = "1.5.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "490aa7465ee685b2ced076bb87ef654a47724a7844e2c7d3af4e749ce5b875dd" +checksum = "90aff65e86db5fe300752551c1b015ef72b708ac54bded8ef43d0d53cb7cb0b1" dependencies = [ "aws-credential-types", "aws-runtime", @@ -617,7 +617,7 @@ dependencies = [ "aws-sdk-ssooidc", "aws-sdk-sts", "aws-smithy-async", - "aws-smithy-http", + "aws-smithy-http 0.61.1", "aws-smithy-json", "aws-smithy-runtime", "aws-smithy-runtime-api", @@ -656,7 +656,7 @@ dependencies = [ "aws-credential-types", "aws-sigv4", "aws-smithy-async", - "aws-smithy-http", + "aws-smithy-http 0.60.12", "aws-smithy-runtime", "aws-smithy-runtime-api", "aws-smithy-types", @@ -674,14 +674,14 @@ dependencies = [ [[package]] name = "aws-sdk-sso" -version = "1.60.0" +version = "1.61.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "60186fab60b24376d3e33b9ff0a43485f99efd470e3b75a9160c849741d63d56" +checksum = "e65ff295979977039a25f5a0bf067a64bc5e6aa38f3cef4037cf42516265553c" dependencies = [ "aws-credential-types", "aws-runtime", "aws-smithy-async", - "aws-smithy-http", + "aws-smithy-http 0.61.1", "aws-smithy-json", "aws-smithy-runtime", "aws-smithy-runtime-api", @@ -696,14 +696,14 @@ dependencies = [ [[package]] name = "aws-sdk-ssooidc" -version = "1.61.0" +version = "1.62.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7033130ce1ee13e6018905b7b976c915963755aef299c1521897679d6cd4f8ef" +checksum = "91430a60f754f235688387b75ee798ef00cfd09709a582be2b7525ebb5306d4f" dependencies = [ "aws-credential-types", "aws-runtime", "aws-smithy-async", - "aws-smithy-http", + "aws-smithy-http 0.61.1", "aws-smithy-json", "aws-smithy-runtime", "aws-smithy-runtime-api", @@ -718,14 +718,14 @@ dependencies = [ [[package]] name = "aws-sdk-sts" -version = "1.61.0" +version = "1.62.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c5c1cac7677179d622b4448b0d31bcb359185295dc6fca891920cfb17e2b5156" +checksum = "9276e139d39fff5a0b0c984fc2d30f970f9a202da67234f948fda02e5bea1dbe" dependencies = [ "aws-credential-types", "aws-runtime", "aws-smithy-async", - "aws-smithy-http", + "aws-smithy-http 0.61.1", "aws-smithy-json", "aws-smithy-query", "aws-smithy-runtime", @@ -746,7 +746,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9bfe75fad52793ce6dec0dc3d4b1f388f038b5eb866c8d4d7f3a8e21b5ea5051" dependencies = [ "aws-credential-types", - "aws-smithy-http", + "aws-smithy-http 0.60.12", "aws-smithy-runtime-api", "aws-smithy-types", "bytes", @@ -793,6 +793,26 @@ dependencies = [ "tracing", ] +[[package]] +name = "aws-smithy-http" +version = "0.61.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e6f276f21c7921fe902826618d1423ae5bf74cf8c1b8472aee8434f3dfd31824" +dependencies = [ + "aws-smithy-runtime-api", + "aws-smithy-types", + "bytes", + "bytes-utils", + "futures-core", + "http 0.2.12", + "http-body 0.4.6", + "once_cell", + "percent-encoding", + "pin-project-lite", + "pin-utils", + "tracing", +] + [[package]] name = "aws-smithy-json" version = "0.61.2" @@ -819,7 +839,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d526a12d9ed61fadefda24abe2e682892ba288c2018bcb38b1b4c111d13f6d92" dependencies = [ "aws-smithy-async", - "aws-smithy-http", + "aws-smithy-http 0.60.12", "aws-smithy-runtime-api", "aws-smithy-types", "bytes", diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index 900d171cb0d4..47471d3808f8 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -37,7 +37,7 @@ backtrace = ["datafusion/backtrace"] [dependencies] arrow = { workspace = true } async-trait = { workspace = true } -aws-config = "1.5.17" +aws-config = "1.5.18" aws-credential-types = "1.2.0" clap = { version = "4.5.30", features = ["derive", "cargo"] } datafusion = { workspace = true, features = [ From 7ef8c0235ff85ea61a62c679c2693030fb8329f1 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 6 Mar 2025 06:41:50 -0500 Subject: [PATCH 16/33] chore(deps): bump bytes from 1.10.0 to 1.10.1 (#15042) Bumps [bytes](https://github.com/tokio-rs/bytes) from 1.10.0 to 1.10.1. - [Release notes](https://github.com/tokio-rs/bytes/releases) - [Changelog](https://github.com/tokio-rs/bytes/blob/master/CHANGELOG.md) - [Commits](https://github.com/tokio-rs/bytes/compare/v1.10.0...v1.10.1) --- updated-dependencies: - dependency-name: bytes dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 6e2a772a1370..3cad2fd73d50 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1217,9 +1217,9 @@ checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" [[package]] name = "bytes" -version = "1.10.0" +version = "1.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f61dac84819c6588b558454b194026eb1f09c293b9036ae9b159e74e73ab6cf9" +checksum = "d71b6127be86fdcfddb610f7182ac57211d4b18a3e9c82eb2d17662f2227ad6a" [[package]] name = "bytes-utils" From da4293323032e2408c9e3b9b28e644a96aea0f13 Mon Sep 17 00:00:00 2001 From: YuNing Chen Date: Thu, 6 Mar 2025 19:48:43 +0800 Subject: [PATCH 17/33] Deprecate `raw_data` method in ScalarValue (#15016) --- datafusion/common/src/scalar/mod.rs | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/datafusion/common/src/scalar/mod.rs b/datafusion/common/src/scalar/mod.rs index 9059ae07e648..367f359ae742 100644 --- a/datafusion/common/src/scalar/mod.rs +++ b/datafusion/common/src/scalar/mod.rs @@ -2764,8 +2764,10 @@ impl ScalarValue { Ok(scalars) } - // TODO: Support more types after other ScalarValue is wrapped with ArrayRef - /// Get raw data (inner array) inside ScalarValue + #[deprecated( + since = "46.0.0", + note = "This function is obsolete. Use `to_array` instead" + )] pub fn raw_data(&self) -> Result { match self { ScalarValue::List(arr) => Ok(arr.to_owned()), From 06be63ffbe1cce242aab43587af3a10ea28c7798 Mon Sep 17 00:00:00 2001 From: Christian <9384305+ctsk@users.noreply.github.com> Date: Thu, 6 Mar 2025 14:03:20 +0100 Subject: [PATCH 18/33] docs: Improve docs on AggregateFunctionExpr construction (#15044) * doc: link to AggregateFunctionBuilder from AggregateFunctionExpr * doc: Alias is required in Builder before construction * cargo fmt * minor: Improve error msg when AggregateExprBuilder::alias is missing * cargo fmt --- datafusion/physical-expr/src/aggregate.rs | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-expr/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs index 07a98340dbe7..34c4e52d517e 100644 --- a/datafusion/physical-expr/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -91,6 +91,9 @@ impl AggregateExprBuilder { } } + /// Constructs an `AggregateFunctionExpr` from the builder + /// + /// Note that an [`Self::alias`] must be provided before calling this method. pub fn build(self) -> Result { let Self { fun, @@ -132,7 +135,11 @@ impl AggregateExprBuilder { let data_type = fun.return_type(&input_exprs_types)?; let is_nullable = fun.is_nullable(); let name = match alias { - None => return internal_err!("alias should be provided"), + None => { + return internal_err!( + "AggregateExprBuilder::alias must be provided prior to calling build" + ) + } Some(alias) => alias, }; @@ -199,6 +206,8 @@ impl AggregateExprBuilder { } /// Physical aggregate expression of a UDAF. +/// +/// Instances are constructed via [`AggregateExprBuilder`]. #[derive(Debug, Clone)] pub struct AggregateFunctionExpr { fun: AggregateUDF, From 87f1e9180d0faf3679468e79d7a1f5c4b2d94c91 Mon Sep 17 00:00:00 2001 From: YuNing Chen Date: Thu, 6 Mar 2025 22:19:10 +0800 Subject: [PATCH 19/33] chore: cleanup deprecated API since version 40 or earlier (#15027) One exception is the optmizer part, which will be addressed in another commit --- datafusion/common/src/dfschema.rs | 13 -- .../core/src/execution/session_state.rs | 115 ------------------ .../datasource-parquet/src/file_format.rs | 15 --- datafusion/expr/src/expr.rs | 26 +--- datafusion/expr/src/logical_plan/extension.rs | 18 --- datafusion/expr/src/utils.rs | 10 -- 6 files changed, 1 insertion(+), 196 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index 99fb179c76a3..b7101e2bbf40 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -159,22 +159,9 @@ impl DFSchema { } /// Create a new `DFSchema` from a list of Arrow [Field]s - #[allow(deprecated)] pub fn from_unqualified_fields( fields: Fields, metadata: HashMap, - ) -> Result { - Self::from_unqualifed_fields(fields, metadata) - } - - /// Create a new `DFSchema` from a list of Arrow [Field]s - #[deprecated( - since = "40.0.0", - note = "Please use `from_unqualified_fields` instead (this one's name is a typo). This method is subject to be removed soon" - )] - pub fn from_unqualifed_fields( - fields: Fields, - metadata: HashMap, ) -> Result { let field_count = fields.len(); let schema = Arc::new(Schema::new_with_metadata(fields, metadata)); diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index d26d5a219e18..0e83156ab53f 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -280,22 +280,6 @@ impl SessionState { .build() } - /// Returns new [`SessionState`] using the provided - /// [`SessionConfig`], [`RuntimeEnv`], and [`CatalogProviderList`] - #[deprecated(since = "40.0.0", note = "Use SessionStateBuilder")] - pub fn new_with_config_rt_and_catalog_list( - config: SessionConfig, - runtime: Arc, - catalog_list: Arc, - ) -> Self { - SessionStateBuilder::new() - .with_config(config) - .with_runtime_env(runtime) - .with_catalog_list(catalog_list) - .with_default_features() - .build() - } - pub(crate) fn resolve_table_ref( &self, table_ref: impl Into, @@ -334,53 +318,6 @@ impl SessionState { }) } - #[deprecated(since = "40.0.0", note = "Use SessionStateBuilder")] - /// Replace the random session id. - pub fn with_session_id(mut self, session_id: String) -> Self { - self.session_id = session_id; - self - } - - #[deprecated(since = "40.0.0", note = "Use SessionStateBuilder")] - /// override default query planner with `query_planner` - pub fn with_query_planner( - mut self, - query_planner: Arc, - ) -> Self { - self.query_planner = query_planner; - self - } - - #[deprecated(since = "40.0.0", note = "Use SessionStateBuilder")] - /// Override the [`AnalyzerRule`]s optimizer plan rules. - pub fn with_analyzer_rules( - mut self, - rules: Vec>, - ) -> Self { - self.analyzer = Analyzer::with_rules(rules); - self - } - - #[deprecated(since = "40.0.0", note = "Use SessionStateBuilder")] - /// Replace the entire list of [`OptimizerRule`]s used to optimize plans - pub fn with_optimizer_rules( - mut self, - rules: Vec>, - ) -> Self { - self.optimizer = Optimizer::with_rules(rules); - self - } - - #[deprecated(since = "40.0.0", note = "Use SessionStateBuilder")] - /// Replace the entire list of [`PhysicalOptimizerRule`]s used to optimize plans - pub fn with_physical_optimizer_rules( - mut self, - physical_optimizers: Vec>, - ) -> Self { - self.physical_optimizers = PhysicalOptimizer::with_rules(physical_optimizers); - self - } - /// Add `analyzer_rule` to the end of the list of /// [`AnalyzerRule`]s used to rewrite queries. pub fn add_analyzer_rule( @@ -391,17 +328,6 @@ impl SessionState { self } - #[deprecated(since = "40.0.0", note = "Use SessionStateBuilder")] - /// Add `optimizer_rule` to the end of the list of - /// [`OptimizerRule`]s used to rewrite queries. - pub fn add_optimizer_rule( - mut self, - optimizer_rule: Arc, - ) -> Self { - self.optimizer.rules.push(optimizer_rule); - self - } - // the add_optimizer_rule takes an owned reference // it should probably be renamed to `with_optimizer_rule` to follow builder style // and `add_optimizer_rule` that takes &mut self added instead of this @@ -412,52 +338,11 @@ impl SessionState { self.optimizer.rules.push(optimizer_rule); } - #[deprecated(since = "40.0.0", note = "Use SessionStateBuilder")] - /// Add `physical_optimizer_rule` to the end of the list of - /// [`PhysicalOptimizerRule`]s used to rewrite queries. - pub fn add_physical_optimizer_rule( - mut self, - physical_optimizer_rule: Arc, - ) -> Self { - self.physical_optimizers.rules.push(physical_optimizer_rule); - self - } - - #[deprecated(since = "40.0.0", note = "Use SessionStateBuilder")] - /// Adds a new [`ConfigExtension`] to TableOptions - pub fn add_table_options_extension( - mut self, - extension: T, - ) -> Self { - self.table_options.extensions.insert(extension); - self - } - - #[deprecated(since = "40.0.0", note = "Use SessionStateBuilder")] - /// Registers a [`FunctionFactory`] to handle `CREATE FUNCTION` statements - pub fn with_function_factory( - mut self, - function_factory: Arc, - ) -> Self { - self.function_factory = Some(function_factory); - self - } - /// Registers a [`FunctionFactory`] to handle `CREATE FUNCTION` statements pub fn set_function_factory(&mut self, function_factory: Arc) { self.function_factory = Some(function_factory); } - #[deprecated(since = "40.0.0", note = "Use SessionStateBuilder")] - /// Replace the extension [`SerializerRegistry`] - pub fn with_serializer_registry( - mut self, - registry: Arc, - ) -> Self { - self.serializer_registry = registry; - self - } - /// Get the function factory pub fn function_factory(&self) -> Option<&Arc> { self.function_factory.as_ref() diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index 268bbf453505..48761d85e708 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -802,21 +802,6 @@ fn get_col_stats( .collect() } -/// Deprecated -/// Use [`statistics_from_parquet_meta_calc`] instead. -/// This method was deprecated because it didn't need to be async so a new method was created -/// that exposes a synchronous API. -#[deprecated( - since = "40.0.0", - note = "please use `statistics_from_parquet_meta_calc` instead" -)] -pub async fn statistics_from_parquet_meta( - metadata: &ParquetMetaData, - table_schema: SchemaRef, -) -> Result { - statistics_from_parquet_meta_calc(metadata, table_schema) -} - fn summarize_min_max_null_counts( min_accs: &mut [Option], max_accs: &mut [Option], diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 3323ea1614fd..56279632251b 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -25,7 +25,6 @@ use std::sync::Arc; use crate::expr_fn::binary_expr; use crate::logical_plan::Subquery; -use crate::utils::expr_to_columns; use crate::Volatility; use crate::{udaf, ExprSchemable, Operator, Signature, WindowFrame, WindowUDF}; @@ -35,7 +34,7 @@ use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeContainer, TreeNodeRecursion, }; use datafusion_common::{ - plan_err, Column, DFSchema, HashMap, Result, ScalarValue, Spans, TableReference, + Column, DFSchema, HashMap, Result, ScalarValue, Spans, TableReference, }; use datafusion_functions_window_common::field::WindowUDFFieldArgs; use sqlparser::ast::{ @@ -1090,11 +1089,6 @@ impl PlannedReplaceSelectItem { } impl Expr { - #[deprecated(since = "40.0.0", note = "use schema_name instead")] - pub fn display_name(&self) -> Result { - Ok(self.schema_name().to_string()) - } - /// The name of the column (field) that this `Expr` will produce. /// /// For example, for a projection (e.g. `SELECT `) the resulting arrow @@ -1444,15 +1438,6 @@ impl Expr { Box::new(high), )) } - - #[deprecated(since = "39.0.0", note = "use try_as_col instead")] - pub fn try_into_col(&self) -> Result { - match self { - Expr::Column(it) => Ok(it.clone()), - _ => plan_err!("Could not coerce '{self}' into Column!"), - } - } - /// Return a reference to the inner `Column` if any /// /// returns `None` if the expression is not a `Column` @@ -1495,15 +1480,6 @@ impl Expr { } } - /// Return all referenced columns of this expression. - #[deprecated(since = "40.0.0", note = "use Expr::column_refs instead")] - pub fn to_columns(&self) -> Result> { - let mut using_columns = HashSet::new(); - expr_to_columns(self, &mut using_columns)?; - - Ok(using_columns) - } - /// Return all references to columns in this expression. /// /// # Example diff --git a/datafusion/expr/src/logical_plan/extension.rs b/datafusion/expr/src/logical_plan/extension.rs index be7153cc4eaa..5bf64a36a654 100644 --- a/datafusion/expr/src/logical_plan/extension.rs +++ b/datafusion/expr/src/logical_plan/extension.rs @@ -82,17 +82,6 @@ pub trait UserDefinedLogicalNode: fmt::Debug + Send + Sync { /// For example: `TopK: k=10` fn fmt_for_explain(&self, f: &mut fmt::Formatter) -> fmt::Result; - #[deprecated(since = "39.0.0", note = "use with_exprs_and_inputs instead")] - #[allow(clippy::wrong_self_convention)] - fn from_template( - &self, - exprs: &[Expr], - inputs: &[LogicalPlan], - ) -> Arc { - self.with_exprs_and_inputs(exprs.to_vec(), inputs.to_vec()) - .unwrap() - } - /// Create a new `UserDefinedLogicalNode` with the specified children /// and expressions. This function is used during optimization /// when the plan is being rewritten and a new instance of the @@ -282,13 +271,6 @@ pub trait UserDefinedLogicalNodeCore: /// For example: `TopK: k=10` fn fmt_for_explain(&self, f: &mut fmt::Formatter) -> fmt::Result; - #[deprecated(since = "39.0.0", note = "use with_exprs_and_inputs instead")] - #[allow(clippy::wrong_self_convention)] - fn from_template(&self, exprs: &[Expr], inputs: &[LogicalPlan]) -> Self { - self.with_exprs_and_inputs(exprs.to_vec(), inputs.to_vec()) - .unwrap() - } - /// Create a new `UserDefinedLogicalNode` with the specified children /// and expressions. This function is used during optimization /// when the plan is being rewritten and a new instance of the diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 3846566e27fe..3404cce17188 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -48,16 +48,6 @@ pub use datafusion_functions_aggregate_common::order::AggregateOrderSensitivity; /// `COUNT()` expressions pub use datafusion_common::utils::expr::COUNT_STAR_EXPANSION; -/// Recursively walk a list of expression trees, collecting the unique set of columns -/// referenced in the expression -#[deprecated(since = "40.0.0", note = "Expr::add_column_refs instead")] -pub fn exprlist_to_columns(expr: &[Expr], accum: &mut HashSet) -> Result<()> { - for e in expr { - expr_to_columns(e, accum)?; - } - Ok(()) -} - /// Count the number of distinct exprs in a list of group by expressions. If the /// first element is a `GroupingSet` expression then it must be the only expr. pub fn grouping_set_expr_count(group_expr: &[Expr]) -> Result { From 986be19dcdae3cbd6acc0fb91202c445a71cf037 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 6 Mar 2025 10:39:56 -0500 Subject: [PATCH 20/33] Implement tree explain for `DataSourceExec` (#15029) * Implement tree explain for DataSourceExec * improve test * Apply suggestions from code review Co-authored-by: Oleks V * fmt --------- Co-authored-by: Oleks V --- datafusion/datasource-csv/src/source.rs | 9 +- datafusion/datasource-parquet/src/source.rs | 11 +- datafusion/datasource/src/file_scan_config.rs | 5 +- datafusion/datasource/src/memory.rs | 30 +-- datafusion/datasource/src/source.rs | 3 +- .../sqllogictest/test_files/explain_tree.slt | 223 ++++++++++++++++-- 6 files changed, 237 insertions(+), 44 deletions(-) diff --git a/datafusion/datasource-csv/src/source.rs b/datafusion/datasource-csv/src/source.rs index 124b32d4f181..bb584433d1a4 100644 --- a/datafusion/datasource-csv/src/source.rs +++ b/datafusion/datasource-csv/src/source.rs @@ -617,8 +617,13 @@ impl FileSource for CsvSource { fn file_type(&self) -> &str { "csv" } - fn fmt_extra(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, ", has_header={}", self.has_header) + fn fmt_extra(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!(f, ", has_header={}", self.has_header) + } + DisplayFormatType::TreeRender => Ok(()), + } } } diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index 51b38f04fa95..683d62a1df49 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -554,14 +554,11 @@ impl FileSource for ParquetSource { fn fmt_extra(&self, t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { match t { - DisplayFormatType::Default - | DisplayFormatType::Verbose - | DisplayFormatType::TreeRender => { + DisplayFormatType::Default | DisplayFormatType::Verbose => { let predicate_string = self .predicate() .map(|p| format!(", predicate={p}")) .unwrap_or_default(); - let pruning_predicate_string = self .pruning_predicate() .map(|pre| { @@ -581,6 +578,12 @@ impl FileSource for ParquetSource { write!(f, "{}{}", predicate_string, pruning_predicate_string) } + DisplayFormatType::TreeRender => { + if let Some(predicate) = self.predicate() { + writeln!(f, "predicate={predicate}")?; + } + Ok(()) + } } } } diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 66ef6262688e..91b5f0157739 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -218,7 +218,10 @@ impl DataSource for FileScanConfig { self.fmt_file_source(t, f) } DisplayFormatType::TreeRender => { - // TODO: collect info + writeln!(f, "format={}", self.file_source.file_type())?; + self.file_source.fmt_extra(t, f)?; + let num_files = self.file_groups.iter().map(Vec::len).sum::(); + writeln!(f, "files={num_files}")?; Ok(()) } } diff --git a/datafusion/datasource/src/memory.rs b/datafusion/datasource/src/memory.rs index 1ea0f2ea2e8b..64fd56971b29 100644 --- a/datafusion/datasource/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -425,25 +425,17 @@ impl DataSource for MemorySourceConfig { } } DisplayFormatType::TreeRender => { - let partition_sizes: Vec<_> = - self.partitions.iter().map(|b| b.len()).collect(); - writeln!(f, "partition_sizes={:?}", partition_sizes)?; - - if let Some(output_ordering) = self.sort_information.first() { - writeln!(f, "output_ordering={}", output_ordering)?; - } - - let eq_properties = self.eq_properties(); - let constraints = eq_properties.constraints(); - if !constraints.is_empty() { - writeln!(f, "constraints={}", constraints)?; - } - - if let Some(limit) = self.fetch { - writeln!(f, "fetch={}", limit)?; - } - - write!(f, "partitions={}", partition_sizes.len()) + let total_rows = self.partitions.iter().map(|b| b.len()).sum::(); + let total_bytes: usize = self + .partitions + .iter() + .flatten() + .map(|batch| batch.get_array_memory_size()) + .sum(); + writeln!(f, "format=memory")?; + writeln!(f, "rows={total_rows}")?; + writeln!(f, "bytes={total_bytes}")?; + Ok(()) } } } diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index bb9790e875b9..6e78df760dc3 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -52,6 +52,7 @@ pub trait DataSource: Send + Sync + Debug { context: Arc, ) -> datafusion_common::Result; fn as_any(&self) -> &dyn Any; + /// Format this source for display in explain plans fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result; /// Return a copy of this DataSource with a new partitioning scheme @@ -103,7 +104,7 @@ impl DisplayAs for DataSourceExec { DisplayFormatType::Default | DisplayFormatType::Verbose => { write!(f, "DataSourceExec: ")?; } - DisplayFormatType::TreeRender => write!(f, "")?, + DisplayFormatType::TreeRender => {} } self.data_source.fmt_as(t, f) } diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index 18de2c6a617f..9d50b9bd626e 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -54,10 +54,36 @@ STORED AS PARQUET LOCATION 'test_files/scratch/explain_tree/table2.parquet'; -# table3: Memoru +# table3: Memory statement ok CREATE TABLE table3 as select * from table1; +# table4: JSON +query I +COPY (SELECT * from table1) +TO 'test_files/scratch/explain_tree/table4.json' +---- +3 + +statement ok +CREATE EXTERNAL TABLE table4 +STORED AS JSON +LOCATION 'test_files/scratch/explain_tree/table4.json'; + +# table5: ARROW +query I +COPY (SELECT * from table1) +TO 'test_files/scratch/explain_tree/table5.arrow' +---- +3 + +statement ok +CREATE EXTERNAL TABLE table5 +STORED AS ARROW +LOCATION 'test_files/scratch/explain_tree/table5.arrow'; + + + ######## Begin Queries ######## # Filter @@ -83,7 +109,10 @@ physical_plan 12)└─────────────┬─────────────┘ 13)┌─────────────┴─────────────┐ 14)│ DataSourceExec │ -15)└───────────────────────────┘ +15)│ -------------------- │ +16)│ files: 1 │ +17)│ format: csv │ +18)└───────────────────────────┘ # Aggregate query TT @@ -110,7 +139,10 @@ physical_plan 15)└─────────────┬─────────────┘ 16)┌─────────────┴─────────────┐ 17)│ DataSourceExec │ -18)└───────────────────────────┘ +18)│ -------------------- │ +19)│ files: 1 │ +20)│ format: csv │ +21)└───────────────────────────┘ # 2 Joins query TT @@ -139,7 +171,10 @@ physical_plan 15)└─────────────┬─────────────┘└─────────────┬─────────────┘ 16)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ 17)│ DataSourceExec ││ DataSourceExec │ -18)└───────────────────────────┘└───────────────────────────┘ +18)│ -------------------- ││ -------------------- │ +19)│ files: 1 ││ files: 1 │ +20)│ format: csv ││ format: parquet │ +21)└───────────────────────────┘└───────────────────────────┘ # 3 Joins query TT @@ -175,18 +210,22 @@ physical_plan 13)┌─────────────┴─────────────┐┌─────────────┴─────────────┐┌─────────────┴─────────────┐ 14)│ CoalesceBatchesExec ││ CoalesceBatchesExec ││ DataSourceExec │ 15)│ ││ ││ -------------------- │ -16)│ ││ ││ partition_sizes: [1] │ -17)│ ││ ││ partitions: 1 │ -18)└─────────────┬─────────────┘└─────────────┬─────────────┘└───────────────────────────┘ -19)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -20)│ RepartitionExec ││ RepartitionExec │ -21)└─────────────┬─────────────┘└─────────────┬─────────────┘ -22)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -23)│ RepartitionExec ││ RepartitionExec │ -24)└─────────────┬─────────────┘└─────────────┬─────────────┘ -25)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -26)│ DataSourceExec ││ DataSourceExec │ -27)└───────────────────────────┘└───────────────────────────┘ +16)│ ││ ││ bytes: 1560 │ +17)│ ││ ││ format: memory │ +18)│ ││ ││ rows: 1 │ +19)└─────────────┬─────────────┘└─────────────┬─────────────┘└───────────────────────────┘ +20)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +21)│ RepartitionExec ││ RepartitionExec │ +22)└─────────────┬─────────────┘└─────────────┬─────────────┘ +23)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +24)│ RepartitionExec ││ RepartitionExec │ +25)└─────────────┬─────────────┘└─────────────┬─────────────┘ +26)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +27)│ DataSourceExec ││ DataSourceExec │ +28)│ -------------------- ││ -------------------- │ +29)│ files: 1 ││ files: 1 │ +30)│ format: csv ││ format: parquet │ +31)└───────────────────────────┘└───────────────────────────┘ # Long Filter (demonstrate what happens with wrapping) query TT @@ -213,9 +252,153 @@ physical_plan 12)└─────────────┬─────────────┘ 13)┌─────────────┴─────────────┐ 14)│ DataSourceExec │ -15)└───────────────────────────┘ +15)│ -------------------- │ +16)│ files: 1 │ +17)│ format: csv │ +18)└───────────────────────────┘ + +# Query with filter on csv +query TT +explain SELECT int_col FROM table1 WHERE string_col != 'foo'; +---- +logical_plan +01)Projection: table1.int_col +02)--Filter: table1.string_col != Utf8("foo") +03)----TableScan: table1 projection=[int_col, string_col], partial_filters=[table1.string_col != Utf8("foo")] +physical_plan +01)┌───────────────────────────┐ +02)│ CoalesceBatchesExec │ +03)└─────────────┬─────────────┘ +04)┌─────────────┴─────────────┐ +05)│ FilterExec │ +06)│ -------------------- │ +07)│ predicate: │ +08)│ string_col@1 != foo │ +09)└─────────────┬─────────────┘ +10)┌─────────────┴─────────────┐ +11)│ RepartitionExec │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ DataSourceExec │ +15)│ -------------------- │ +16)│ files: 1 │ +17)│ format: csv │ +18)└───────────────────────────┘ +# Query with filter on parquet +query TT +explain SELECT int_col FROM table2 WHERE string_col != 'foo'; +---- +logical_plan +01)Projection: table2.int_col +02)--Filter: table2.string_col != Utf8View("foo") +03)----TableScan: table2 projection=[int_col, string_col], partial_filters=[table2.string_col != Utf8View("foo")] +physical_plan +01)┌───────────────────────────┐ +02)│ CoalesceBatchesExec │ +03)└─────────────┬─────────────┘ +04)┌─────────────┴─────────────┐ +05)│ FilterExec │ +06)│ -------------------- │ +07)│ predicate: │ +08)│ string_col@1 != foo │ +09)└─────────────┬─────────────┘ +10)┌─────────────┴─────────────┐ +11)│ RepartitionExec │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ DataSourceExec │ +15)│ -------------------- │ +16)│ files: 1 │ +17)│ format: parquet │ +18)│ │ +19)│ predicate: │ +20)│ string_col@1 != foo │ +21)└───────────────────────────┘ + +# Query with filter on memory +query TT +explain SELECT int_col FROM table3 WHERE string_col != 'foo'; +---- +logical_plan +01)Projection: table3.int_col +02)--Filter: table3.string_col != Utf8("foo") +03)----TableScan: table3 projection=[int_col, string_col] +physical_plan +01)┌───────────────────────────┐ +02)│ CoalesceBatchesExec │ +03)└─────────────┬─────────────┘ +04)┌─────────────┴─────────────┐ +05)│ FilterExec │ +06)│ -------------------- │ +07)│ predicate: │ +08)│ string_col@1 != foo │ +09)└─────────────┬─────────────┘ +10)┌─────────────┴─────────────┐ +11)│ DataSourceExec │ +12)│ -------------------- │ +13)│ bytes: 1560 │ +14)│ format: memory │ +15)│ rows: 1 │ +16)└───────────────────────────┘ + +# Query with filter on json +query TT +explain SELECT int_col FROM table4 WHERE string_col != 'foo'; +---- +logical_plan +01)Projection: table4.int_col +02)--Filter: table4.string_col != Utf8("foo") +03)----TableScan: table4 projection=[int_col, string_col], partial_filters=[table4.string_col != Utf8("foo")] +physical_plan +01)┌───────────────────────────┐ +02)│ CoalesceBatchesExec │ +03)└─────────────┬─────────────┘ +04)┌─────────────┴─────────────┐ +05)│ FilterExec │ +06)│ -------------------- │ +07)│ predicate: │ +08)│ string_col@1 != foo │ +09)└─────────────┬─────────────┘ +10)┌─────────────┴─────────────┐ +11)│ RepartitionExec │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ DataSourceExec │ +15)│ -------------------- │ +16)│ files: 1 │ +17)│ format: json │ +18)└───────────────────────────┘ + +# Query with filter on arrow +query TT +explain SELECT int_col FROM table5 WHERE string_col != 'foo'; +---- +logical_plan +01)Projection: table5.int_col +02)--Filter: table5.string_col != Utf8("foo") +03)----TableScan: table5 projection=[int_col, string_col], partial_filters=[table5.string_col != Utf8("foo")] +physical_plan +01)┌───────────────────────────┐ +02)│ CoalesceBatchesExec │ +03)└─────────────┬─────────────┘ +04)┌─────────────┴─────────────┐ +05)│ FilterExec │ +06)│ -------------------- │ +07)│ predicate: │ +08)│ string_col@1 != foo │ +09)└─────────────┬─────────────┘ +10)┌─────────────┴─────────────┐ +11)│ RepartitionExec │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ DataSourceExec │ +15)│ -------------------- │ +16)│ files: 1 │ +17)│ format: arrow │ +18)└───────────────────────────┘ + # cleanup statement ok drop table table1; @@ -225,3 +408,9 @@ drop table table2; statement ok drop table table3; + +statement ok +drop table table4; + +statement ok +drop table table5; From 34efd1fbae39eb0441a43ab976fc23001d1f674a Mon Sep 17 00:00:00 2001 From: Yongting You <2010youy01@gmail.com> Date: Fri, 7 Mar 2025 00:24:22 +0800 Subject: [PATCH 21/33] More comment to aggregation fuzzer (#15048) --- .../aggregation_fuzzer/data_generator.rs | 23 ++++++++++++++++++- .../fuzz_cases/aggregation_fuzzer/mod.rs | 20 ++++++++++++++++ 2 files changed, 42 insertions(+), 1 deletion(-) diff --git a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs index 4d4c6aa79357..54c5744c861b 100644 --- a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs +++ b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs @@ -100,7 +100,28 @@ impl DatasetGeneratorConfig { /// Dataset generator /// -/// It will generate one random [`Dataset`] when `generate` function is called. +/// It will generate random [`Dataset`]s when the `generate` function is called. For each +/// sort key in `sort_keys_set`, an additional sorted dataset will be generated, and the +/// dataset will be chunked into staggered batches. +/// +/// # Example +/// For `DatasetGenerator` with `sort_keys_set = [["a"], ["b"]]`, it will generate 2 +/// datasets. The first one will be sorted by column `a` and get randomly chunked +/// into staggered batches. It might look like the following: +/// ```text +/// a b +/// ---- +/// 1 2 <-- batch 1 +/// 1 1 +/// +/// 2 1 <-- batch 2 +/// +/// 3 3 <-- batch 3 +/// 4 3 +/// 4 1 +/// ``` +/// +/// # Implementation details: /// /// The generation logic in `generate`: /// diff --git a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/mod.rs b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/mod.rs index 7c5b25e4a0e0..1e42ac1f4b30 100644 --- a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/mod.rs +++ b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/mod.rs @@ -15,6 +15,26 @@ // specific language governing permissions and limitations // under the License. +//! Fuzzer for aggregation functions +//! +//! The main idea behind aggregate fuzzing is: for aggregation, DataFusion has many +//! specialized implementations for performance. For example, when the group cardinality +//! is high, DataFusion will skip the first stage of two-stage hash aggregation; when +//! the input is ordered by the group key, there is a separate implementation to perform +//! streaming group by. +//! This fuzzer checks the results of different specialized implementations and +//! ensures their results are consistent. The execution path can be controlled by +//! changing the input ordering or by setting related configuration parameters in +//! `SessionContext`. +//! +//! # Architecture +//! - `aggregate_fuzz.rs` includes the entry point for fuzzer runs. +//! - `QueryBuilder` is used to generate candidate queries. +//! - `DatasetGenerator` is used to generate random datasets. +//! - `SessionContextGenerator` is used to generate `SessionContext` with +//! different configuration parameters to control the execution path of aggregate +//! queries. + use arrow::array::RecordBatch; use arrow::util::pretty::pretty_format_batches; use datafusion::prelude::SessionContext; From 05bb56913ecaeed981c8913cca030af482e6c39b Mon Sep 17 00:00:00 2001 From: Eren Avsarogullari Date: Fri, 7 Mar 2025 02:26:18 -0800 Subject: [PATCH 22/33] feat: Add `array_max` function support (#14470) * Issue-14469 - feat: Add array_max function * Address review comments * Address review comments II * Address review comments III --- datafusion/functions-aggregate/src/min_max.rs | 2 +- datafusion/functions-nested/src/lib.rs | 2 + datafusion/functions-nested/src/max.rs | 137 ++++++++++++++++++ datafusion/sqllogictest/test_files/array.slt | 87 +++++++++++ .../source/user-guide/sql/scalar_functions.md | 33 +++++ 5 files changed, 260 insertions(+), 1 deletion(-) create mode 100644 datafusion/functions-nested/src/max.rs diff --git a/datafusion/functions-aggregate/src/min_max.rs b/datafusion/functions-aggregate/src/min_max.rs index 90fb46883de6..83356e2f9fb4 100644 --- a/datafusion/functions-aggregate/src/min_max.rs +++ b/datafusion/functions-aggregate/src/min_max.rs @@ -573,7 +573,7 @@ fn min_batch(values: &ArrayRef) -> Result { } /// dynamically-typed max(array) -> ScalarValue -fn max_batch(values: &ArrayRef) -> Result { +pub fn max_batch(values: &ArrayRef) -> Result { Ok(match values.data_type() { DataType::Utf8 => { typed_min_max_batch_string!(values, StringArray, Utf8, max_string) diff --git a/datafusion/functions-nested/src/lib.rs b/datafusion/functions-nested/src/lib.rs index 41ebb4366cff..446cd58865c3 100644 --- a/datafusion/functions-nested/src/lib.rs +++ b/datafusion/functions-nested/src/lib.rs @@ -52,6 +52,7 @@ pub mod map; pub mod map_extract; pub mod map_keys; pub mod map_values; +pub mod max; pub mod planner; pub mod position; pub mod range; @@ -144,6 +145,7 @@ pub fn all_default_nested_functions() -> Vec> { length::array_length_udf(), distance::array_distance_udf(), flatten::flatten_udf(), + max::array_max_udf(), sort::array_sort_udf(), repeat::array_repeat_udf(), resize::array_resize_udf(), diff --git a/datafusion/functions-nested/src/max.rs b/datafusion/functions-nested/src/max.rs new file mode 100644 index 000000000000..22bd14740b5e --- /dev/null +++ b/datafusion/functions-nested/src/max.rs @@ -0,0 +1,137 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! [`ScalarUDFImpl`] definitions for array_max function. +use crate::utils::make_scalar_function; +use arrow::array::ArrayRef; +use arrow::datatypes::DataType; +use arrow::datatypes::DataType::List; +use datafusion_common::cast::as_list_array; +use datafusion_common::utils::take_function_args; +use datafusion_common::{exec_err, ScalarValue}; +use datafusion_doc::Documentation; +use datafusion_expr::{ColumnarValue, ScalarUDFImpl, Signature, Volatility}; +use datafusion_functions_aggregate::min_max; +use datafusion_macros::user_doc; +use itertools::Itertools; +use std::any::Any; + +make_udf_expr_and_func!( + ArrayMax, + array_max, + array, + "returns the maximum value in the array.", + array_max_udf +); + +#[user_doc( + doc_section(label = "Array Functions"), + description = "Returns the maximum value in the array.", + syntax_example = "array_max(array)", + sql_example = r#"```sql +> select array_max([3,1,4,2]); ++-----------------------------------------+ +| array_max(List([3,1,4,2])) | ++-----------------------------------------+ +| 4 | ++-----------------------------------------+ +```"#, + argument( + name = "array", + description = "Array expression. Can be a constant, column, or function, and any combination of array operators." + ) +)] +#[derive(Debug)] +pub struct ArrayMax { + signature: Signature, + aliases: Vec, +} + +impl Default for ArrayMax { + fn default() -> Self { + Self::new() + } +} + +impl ArrayMax { + pub fn new() -> Self { + Self { + signature: Signature::array(Volatility::Immutable), + aliases: vec!["list_max".to_string()], + } + } +} + +impl ScalarUDFImpl for ArrayMax { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "array_max" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, arg_types: &[DataType]) -> datafusion_common::Result { + match &arg_types[0] { + List(field) => Ok(field.data_type().clone()), + _ => exec_err!("Not reachable, data_type should be List"), + } + } + + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> datafusion_common::Result { + make_scalar_function(array_max_inner)(args) + } + + fn aliases(&self) -> &[String] { + &self.aliases + } + + fn documentation(&self) -> Option<&Documentation> { + self.doc() + } +} + +/// array_max SQL function +/// +/// There is one argument for array_max as the array. +/// `array_max(array)` +/// +/// For example: +/// > array_max(\[1, 3, 2]) -> 3 +pub fn array_max_inner(args: &[ArrayRef]) -> datafusion_common::Result { + let [arg1] = take_function_args("array_max", args)?; + + match arg1.data_type() { + List(_) => { + let input_list_array = as_list_array(&arg1)?; + let result_vec = input_list_array + .iter() + .flat_map(|arr| min_max::max_batch(&arr.unwrap())) + .collect_vec(); + ScalarValue::iter_to_array(result_vec) + } + _ => exec_err!("array_max does not support type: {:?}", arg1.data_type()), + } +} diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index c8f6a985bb22..3b7f12960681 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -1435,6 +1435,93 @@ NULL 23 NULL 43 5 NULL + +## array_max +# array_max scalar function #1 (with positive index) +query I +select array_max(make_array(5, 3, 6, 4)); +---- +6 + +query I +select array_max(make_array(5, 3, 4, NULL, 6, NULL)); +---- +6 + +query I +select array_max(make_array(NULL, NULL)); +---- +NULL + +query T +select array_max(make_array('h', 'e', 'o', 'l', 'l')); +---- +o + +query T +select array_max(make_array('h', 'e', 'l', NULL, 'l', 'o', NULL)); +---- +o + +query B +select array_max(make_array(false, true, false, true)); +---- +true + +query B +select array_max(make_array(false, true, NULL, false, true)); +---- +true + +query D +select array_max(make_array(DATE '1992-09-01', DATE '1993-03-01', DATE '1999-05-01', DATE '1985-11-01')); +---- +1999-05-01 + +query D +select array_max(make_array(DATE '1995-09-01', DATE '1999-05-01', DATE '1993-03-01', NULL)); +---- +1999-05-01 + +query P +select array_max(make_array(TIMESTAMP '1992-09-01', TIMESTAMP '1995-06-01', TIMESTAMP '1984-10-01')); +---- +1995-06-01T00:00:00 + +query P +select array_max(make_array(NULL, TIMESTAMP '1996-10-01', TIMESTAMP '1995-06-01')); +---- +1996-10-01T00:00:00 + +query R +select array_max(make_array(5.1, -3.2, 6.3, 4.9)); +---- +6.3 + +query ?I +select input, array_max(input) from (select make_array(d - 1, d, d + 1) input from (values (0), (10), (20), (30), (NULL)) t(d)) +---- +[-1, 0, 1] 1 +[9, 10, 11] 11 +[19, 20, 21] 21 +[29, 30, 31] 31 +[NULL, NULL, NULL] NULL + +query II +select array_max(arrow_cast(make_array(1, 2, 3), 'FixedSizeList(3, Int64)')), array_max(arrow_cast(make_array(1), 'FixedSizeList(1, Int64)')); +---- +3 1 + +query I +select array_max(make_array()); +---- +NULL + +# Testing with empty arguments should result in an error +query error DataFusion error: Error during planning: 'array_max' does not support zero arguments +select array_max(); + + ## array_pop_back (aliases: `list_pop_back`) # array_pop_back scalar function with null diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index fb4043c33efc..60ecf7bd78d4 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -2524,6 +2524,7 @@ _Alias of [current_date](#current_date)._ - [array_intersect](#array_intersect) - [array_join](#array_join) - [array_length](#array_length) +- [array_max](#array_max) - [array_ndims](#array_ndims) - [array_pop_back](#array_pop_back) - [array_pop_front](#array_pop_front) @@ -2569,6 +2570,7 @@ _Alias of [current_date](#current_date)._ - [list_intersect](#list_intersect) - [list_join](#list_join) - [list_length](#list_length) +- [list_max](#list_max) - [list_ndims](#list_ndims) - [list_pop_back](#list_pop_back) - [list_pop_front](#list_pop_front) @@ -3002,6 +3004,33 @@ array_length(array, dimension) - list_length +### `array_max` + +Returns the maximum value in the array. + +```sql +array_max(array) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. + +#### Example + +```sql +> select array_max([3,1,4,2]); ++-----------------------------------------+ +| array_max(List([3,1,4,2])) | ++-----------------------------------------+ +| 4 | ++-----------------------------------------+ +``` + +#### Aliases + +- list_max + ### `array_ndims` Returns the number of dimensions of the array. @@ -3759,6 +3788,10 @@ _Alias of [array_to_string](#array_to_string)._ _Alias of [array_length](#array_length)._ +### `list_max` + +_Alias of [array_max](#array_max)._ + ### `list_ndims` _Alias of [array_ndims](#array_ndims)._ From 297af9533b7779b7d4bf847b8cea48e739bc16cc Mon Sep 17 00:00:00 2001 From: wiedld Date: Fri, 7 Mar 2025 05:57:47 -0800 Subject: [PATCH 23/33] refactor(15003): refactor test suite in EnforceDistribution, to use test config builder (#15010) --- .../enforce_distribution.rs | 657 +++++++++++++----- 1 file changed, 488 insertions(+), 169 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 85d826109f89..f22a896c1894 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -372,46 +372,91 @@ macro_rules! plans_matches_expected { } } +fn test_suite_default_config_options() -> ConfigOptions { + let mut config = ConfigOptions::new(); + + // By default, will not repartition / resort data if it is already sorted. + config.optimizer.prefer_existing_sort = false; + + // By default, will attempt to convert Union to Interleave. + config.optimizer.prefer_existing_union = false; + + // By default, will not repartition file scans. + config.optimizer.repartition_file_scans = false; + config.optimizer.repartition_file_min_size = 1024; + + // By default, set query execution concurrency to 10. + config.execution.target_partitions = 10; + + // Use a small batch size, to trigger RoundRobin in tests + config.execution.batch_size = 1; + + config +} + +/// How the optimizers are run. +#[derive(PartialEq, Clone)] +enum DoFirst { + /// Runs: (EnforceDistribution, EnforceDistribution, EnforceSorting) + Distribution, + /// Runs: (EnforceSorting, EnforceDistribution, EnforceDistribution) + Sorting, +} + +#[derive(Clone)] +struct TestConfig { + config: ConfigOptions, + optimizers_to_run: DoFirst, +} + +impl TestConfig { + fn new(optimizers_to_run: DoFirst) -> Self { + Self { + config: test_suite_default_config_options(), + optimizers_to_run, + } + } + + /// If preferred, will not repartition / resort data if it is already sorted. + fn with_prefer_existing_sort(mut self) -> Self { + self.config.optimizer.prefer_existing_sort = true; + self + } + + /// If preferred, will not attempt to convert Union to Interleave. + fn with_prefer_existing_union(mut self) -> Self { + self.config.optimizer.prefer_existing_union = true; + self + } + + /// If preferred, will repartition file scans. + /// Accepts a minimum file size to repartition. + fn with_prefer_repartition_file_scans(mut self, file_min_size: usize) -> Self { + self.config.optimizer.repartition_file_scans = true; + self.config.optimizer.repartition_file_min_size = file_min_size; + self + } + + /// Set the preferred target partitions for query execution concurrency. + fn with_query_execution_partitions(mut self, target_partitions: usize) -> Self { + self.config.execution.target_partitions = target_partitions; + self + } +} + /// Runs the repartition optimizer and asserts the plan against the expected /// Arguments /// * `EXPECTED_LINES` - Expected output plan /// * `PLAN` - Input plan -/// * `FIRST_ENFORCE_DIST` - -/// true: (EnforceDistribution, EnforceDistribution, EnforceSorting) -/// false: else runs (EnforceSorting, EnforceDistribution, EnforceDistribution) -/// * `PREFER_EXISTING_SORT` (optional) - if true, will not repartition / resort data if it is already sorted -/// * `TARGET_PARTITIONS` (optional) - number of partitions to repartition to -/// * `REPARTITION_FILE_SCANS` (optional) - if true, will repartition file scans -/// * `REPARTITION_FILE_MIN_SIZE` (optional) - minimum file size to repartition -/// * `PREFER_EXISTING_UNION` (optional) - if true, will not attempt to convert Union to Interleave +/// * `CONFIG` - [`TestConfig`] macro_rules! assert_optimized { - ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr) => { - assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, false, 10, false, 1024, false); - }; - - ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $PREFER_EXISTING_SORT: expr) => { - assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, $PREFER_EXISTING_SORT, 10, false, 1024, false); - }; - - ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $PREFER_EXISTING_SORT: expr, $PREFER_EXISTING_UNION: expr) => { - assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, $PREFER_EXISTING_SORT, 10, false, 1024, $PREFER_EXISTING_UNION); - }; - - ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $PREFER_EXISTING_SORT: expr, $TARGET_PARTITIONS: expr, $REPARTITION_FILE_SCANS: expr, $REPARTITION_FILE_MIN_SIZE: expr) => { - assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, $PREFER_EXISTING_SORT, $TARGET_PARTITIONS, $REPARTITION_FILE_SCANS, $REPARTITION_FILE_MIN_SIZE, false); - }; - - ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $PREFER_EXISTING_SORT: expr, $TARGET_PARTITIONS: expr, $REPARTITION_FILE_SCANS: expr, $REPARTITION_FILE_MIN_SIZE: expr, $PREFER_EXISTING_UNION: expr) => { + ($EXPECTED_LINES: expr, $PLAN: expr, $CONFIG: expr) => { let expected_lines: Vec<&str> = $EXPECTED_LINES.iter().map(|s| *s).collect(); - let mut config = ConfigOptions::new(); - config.execution.target_partitions = $TARGET_PARTITIONS; - config.optimizer.repartition_file_scans = $REPARTITION_FILE_SCANS; - config.optimizer.repartition_file_min_size = $REPARTITION_FILE_MIN_SIZE; - config.optimizer.prefer_existing_sort = $PREFER_EXISTING_SORT; - config.optimizer.prefer_existing_union = $PREFER_EXISTING_UNION; - // Use a small batch size, to trigger RoundRobin in tests - config.execution.batch_size = 1; + let TestConfig { + config, + optimizers_to_run, + } = $CONFIG; // NOTE: These tests verify the joint `EnforceDistribution` + `EnforceSorting` cascade // because they were written prior to the separation of `BasicEnforcement` into @@ -455,7 +500,7 @@ macro_rules! assert_optimized { // TODO: End state payloads will be checked here. } - let optimized = if $FIRST_ENFORCE_DIST { + let optimized = if *optimizers_to_run == DoFirst::Distribution { // Run enforce distribution rule first: let optimizer = EnforceDistribution::new(); let optimized = optimizer.optimize(optimized, &config)?; @@ -602,8 +647,12 @@ fn multi_hash_joins() -> Result<()> { " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], }; - assert_optimized!(expected, top_join.clone(), true); - assert_optimized!(expected, top_join, false); + assert_optimized!( + expected, + top_join.clone(), + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!(expected, top_join, &TestConfig::new(DoFirst::Sorting)); } JoinType::RightSemi | JoinType::RightAnti => {} } @@ -666,8 +715,12 @@ fn multi_hash_joins() -> Result<()> { " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], }; - assert_optimized!(expected, top_join.clone(), true); - assert_optimized!(expected, top_join, false); + assert_optimized!( + expected, + top_join.clone(), + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!(expected, top_join, &TestConfig::new(DoFirst::Sorting)); } JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => {} } @@ -723,8 +776,12 @@ fn multi_joins_after_alias() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected, top_join.clone(), true); - assert_optimized!(expected, top_join, false); + assert_optimized!( + expected, + top_join.clone(), + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!(expected, top_join, &TestConfig::new(DoFirst::Sorting)); // Join on (a2 == c) let top_join_on = vec![( @@ -749,8 +806,12 @@ fn multi_joins_after_alias() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected, top_join.clone(), true); - assert_optimized!(expected, top_join, false); + assert_optimized!( + expected, + top_join.clone(), + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!(expected, top_join, &TestConfig::new(DoFirst::Sorting)); Ok(()) } @@ -803,8 +864,12 @@ fn multi_joins_after_multi_alias() -> Result<()> { " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected, top_join.clone(), true); - assert_optimized!(expected, top_join, false); + assert_optimized!( + expected, + top_join.clone(), + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!(expected, top_join, &TestConfig::new(DoFirst::Sorting)); Ok(()) } @@ -843,8 +908,12 @@ fn join_after_agg_alias() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected, join.clone(), true); - assert_optimized!(expected, join, false); + assert_optimized!( + expected, + join.clone(), + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!(expected, join, &TestConfig::new(DoFirst::Sorting)); Ok(()) } @@ -896,8 +965,12 @@ fn hash_join_key_ordering() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected, join.clone(), true); - assert_optimized!(expected, join, false); + assert_optimized!( + expected, + join.clone(), + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!(expected, join, &TestConfig::new(DoFirst::Sorting)); Ok(()) } @@ -1022,8 +1095,16 @@ fn multi_hash_join_key_ordering() -> Result<()> { " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected, filter_top_join.clone(), true); - assert_optimized!(expected, filter_top_join, false); + assert_optimized!( + expected, + filter_top_join.clone(), + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!( + expected, + filter_top_join, + &TestConfig::new(DoFirst::Sorting) + ); Ok(()) } @@ -1301,6 +1382,7 @@ fn reorder_join_keys_to_right_input() -> Result<()> { Ok(()) } +/// These test cases use [`TestConfig::with_prefer_existing_sort`]. #[test] fn multi_smj_joins() -> Result<()> { let left = parquet_exec(); @@ -1402,7 +1484,11 @@ fn multi_smj_joins() -> Result<()> { " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], }; - assert_optimized!(expected, top_join.clone(), true, true); + assert_optimized!( + expected, + top_join.clone(), + &TestConfig::new(DoFirst::Distribution).with_prefer_existing_sort() + ); let expected_first_sort_enforcement = match join_type { // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs @@ -1456,7 +1542,11 @@ fn multi_smj_joins() -> Result<()> { " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], }; - assert_optimized!(expected_first_sort_enforcement, top_join, false, true); + assert_optimized!( + expected_first_sort_enforcement, + top_join, + &TestConfig::new(DoFirst::Sorting).with_prefer_existing_sort() + ); match join_type { JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { @@ -1513,7 +1603,11 @@ fn multi_smj_joins() -> Result<()> { // this match arm cannot be reached _ => unreachable!() }; - assert_optimized!(expected, top_join.clone(), true, true); + assert_optimized!( + expected, + top_join.clone(), + &TestConfig::new(DoFirst::Distribution).with_prefer_existing_sort() + ); let expected_first_sort_enforcement = match join_type { // Should include 6 RepartitionExecs (3 of them preserves order) and 3 SortExecs @@ -1559,7 +1653,12 @@ fn multi_smj_joins() -> Result<()> { // this match arm cannot be reached _ => unreachable!() }; - assert_optimized!(expected_first_sort_enforcement, top_join, false, true); + + assert_optimized!( + expected_first_sort_enforcement, + top_join, + &TestConfig::new(DoFirst::Sorting).with_prefer_existing_sort() + ); } _ => {} } @@ -1568,6 +1667,7 @@ fn multi_smj_joins() -> Result<()> { Ok(()) } +/// These test cases use [`TestConfig::with_prefer_existing_sort`]. #[test] fn smj_join_key_ordering() -> Result<()> { // group by (a as a1, b as b1) @@ -1633,7 +1733,11 @@ fn smj_join_key_ordering() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected, join.clone(), true, true); + assert_optimized!( + expected, + join.clone(), + &TestConfig::new(DoFirst::Distribution).with_prefer_existing_sort() + ); let expected_first_sort_enforcement = &[ "SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)]", @@ -1659,7 +1763,11 @@ fn smj_join_key_ordering() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected_first_sort_enforcement, join, false, true); + assert_optimized!( + expected_first_sort_enforcement, + join, + &TestConfig::new(DoFirst::Sorting).with_prefer_existing_sort() + ); Ok(()) } @@ -1690,7 +1798,7 @@ fn merge_does_not_need_sort() -> Result<()> { " CoalesceBatchesExec: target_batch_size=4096", " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; - assert_optimized!(expected, exec, true); + assert_optimized!(expected, exec, &TestConfig::new(DoFirst::Distribution)); // In this case preserving ordering through order preserving operators is not desirable // (according to flag: PREFER_EXISTING_SORT) @@ -1702,7 +1810,7 @@ fn merge_does_not_need_sort() -> Result<()> { " CoalesceBatchesExec: target_batch_size=4096", " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; - assert_optimized!(expected, exec, false); + assert_optimized!(expected, exec, &TestConfig::new(DoFirst::Sorting)); Ok(()) } @@ -1743,8 +1851,12 @@ fn union_to_interleave() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan.clone(), false); + assert_optimized!( + expected, + plan.clone(), + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!(expected, plan.clone(), &TestConfig::new(DoFirst::Sorting)); Ok(()) } @@ -1786,24 +1898,16 @@ fn union_not_to_interleave() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - // no sort in the plan but since we need it as a parameter, make it default false - let prefer_existing_sort = false; - let first_enforce_distribution = true; - let prefer_existing_union = true; assert_optimized!( expected, plan.clone(), - first_enforce_distribution, - prefer_existing_sort, - prefer_existing_union + &TestConfig::new(DoFirst::Distribution).with_prefer_existing_union() ); assert_optimized!( expected, plan, - !first_enforce_distribution, - prefer_existing_sort, - prefer_existing_union + &TestConfig::new(DoFirst::Sorting).with_prefer_existing_union() ); Ok(()) @@ -1821,8 +1925,12 @@ fn added_repartition_to_single_partition() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); + assert_optimized!( + expected, + plan.clone(), + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!(expected, plan, &TestConfig::new(DoFirst::Sorting)); Ok(()) } @@ -1840,14 +1948,17 @@ fn repartition_deepest_node() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); + assert_optimized!( + expected, + plan.clone(), + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!(expected, plan, &TestConfig::new(DoFirst::Sorting)); Ok(()) } #[test] - fn repartition_unsorted_limit() -> Result<()> { let plan = limit_exec(filter_exec(parquet_exec())); @@ -1861,8 +1972,12 @@ fn repartition_unsorted_limit() -> Result<()> { " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); + assert_optimized!( + expected, + plan.clone(), + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!(expected, plan, &TestConfig::new(DoFirst::Sorting)); Ok(()) } @@ -1883,8 +1998,12 @@ fn repartition_sorted_limit() -> Result<()> { " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); + assert_optimized!( + expected, + plan.clone(), + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!(expected, plan, &TestConfig::new(DoFirst::Sorting)); Ok(()) } @@ -1911,8 +2030,12 @@ fn repartition_sorted_limit_with_filter() -> Result<()> { " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); + assert_optimized!( + expected, + plan.clone(), + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!(expected, plan, &TestConfig::new(DoFirst::Sorting)); Ok(()) } @@ -1941,8 +2064,12 @@ fn repartition_ignores_limit() -> Result<()> { // Expect no repartition to happen for local limit " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); + assert_optimized!( + expected, + plan.clone(), + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!(expected, plan, &TestConfig::new(DoFirst::Sorting)); Ok(()) } @@ -1961,8 +2088,12 @@ fn repartition_ignores_union() -> Result<()> { " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); + assert_optimized!( + expected, + plan.clone(), + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!(expected, plan, &TestConfig::new(DoFirst::Sorting)); Ok(()) } @@ -1982,8 +2113,12 @@ fn repartition_through_sort_preserving_merge() -> Result<()> { "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); + assert_optimized!( + expected, + plan.clone(), + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!(expected, plan, &TestConfig::new(DoFirst::Sorting)); Ok(()) } @@ -2008,14 +2143,18 @@ fn repartition_ignores_sort_preserving_merge() -> Result<()> { " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; - assert_optimized!(expected, plan.clone(), true); + assert_optimized!( + expected, + plan.clone(), + &TestConfig::new(DoFirst::Distribution) + ); let expected = &[ "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", " CoalescePartitionsExec", " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; - assert_optimized!(expected, plan, false); + assert_optimized!(expected, plan, &TestConfig::new(DoFirst::Sorting)); Ok(()) } @@ -2039,7 +2178,11 @@ fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; - assert_optimized!(expected, plan.clone(), true); + assert_optimized!( + expected, + plan.clone(), + &TestConfig::new(DoFirst::Distribution) + ); let expected = &[ "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", @@ -2048,11 +2191,12 @@ fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; - assert_optimized!(expected, plan, false); + assert_optimized!(expected, plan, &TestConfig::new(DoFirst::Sorting)); Ok(()) } +/// These test cases use [`TestConfig::with_prefer_existing_sort`]. #[test] fn repartition_does_not_destroy_sort() -> Result<()> { // SortRequired @@ -2075,8 +2219,16 @@ fn repartition_does_not_destroy_sort() -> Result<()> { " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet", ]; - assert_optimized!(expected, plan.clone(), true, true); - assert_optimized!(expected, plan, false, true); + assert_optimized!( + expected, + plan.clone(), + &TestConfig::new(DoFirst::Distribution).with_prefer_existing_sort() + ); + assert_optimized!( + expected, + plan, + &TestConfig::new(DoFirst::Sorting).with_prefer_existing_sort() + ); Ok(()) } @@ -2116,8 +2268,12 @@ fn repartition_does_not_destroy_sort_more_complex() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); + assert_optimized!( + expected, + plan.clone(), + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!(expected, plan, &TestConfig::new(DoFirst::Sorting)); Ok(()) } @@ -2150,7 +2306,11 @@ fn repartition_transitively_with_projection() -> Result<()> { " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected, plan.clone(), true); + assert_optimized!( + expected, + plan.clone(), + &TestConfig::new(DoFirst::Distribution) + ); let expected_first_sort_enforcement = &[ "SortExec: expr=[sum@0 ASC], preserve_partitioning=[false]", @@ -2160,7 +2320,11 @@ fn repartition_transitively_with_projection() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected_first_sort_enforcement, plan, false); + assert_optimized!( + expected_first_sort_enforcement, + plan, + &TestConfig::new(DoFirst::Sorting) + ); Ok(()) } @@ -2192,8 +2356,12 @@ fn repartition_ignores_transitively_with_projection() -> Result<()> { " ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); + assert_optimized!( + expected, + plan.clone(), + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!(expected, plan, &TestConfig::new(DoFirst::Sorting)); Ok(()) } @@ -2225,8 +2393,12 @@ fn repartition_transitively_past_sort_with_projection() -> Result<()> { " ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); + assert_optimized!( + expected, + plan.clone(), + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!(expected, plan, &TestConfig::new(DoFirst::Sorting)); Ok(()) } @@ -2249,7 +2421,11 @@ fn repartition_transitively_past_sort_with_filter() -> Result<()> { " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected, plan.clone(), true); + assert_optimized!( + expected, + plan.clone(), + &TestConfig::new(DoFirst::Distribution) + ); let expected_first_sort_enforcement = &[ "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", @@ -2259,7 +2435,11 @@ fn repartition_transitively_past_sort_with_filter() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected_first_sort_enforcement, plan, false); + assert_optimized!( + expected_first_sort_enforcement, + plan, + &TestConfig::new(DoFirst::Sorting) + ); Ok(()) } @@ -2296,7 +2476,11 @@ fn repartition_transitively_past_sort_with_projection_and_filter() -> Result<()> " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected, plan.clone(), true); + assert_optimized!( + expected, + plan.clone(), + &TestConfig::new(DoFirst::Distribution) + ); let expected_first_sort_enforcement = &[ "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", @@ -2306,7 +2490,11 @@ fn repartition_transitively_past_sort_with_projection_and_filter() -> Result<()> " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected_first_sort_enforcement, plan, false); + assert_optimized!( + expected_first_sort_enforcement, + plan, + &TestConfig::new(DoFirst::Sorting) + ); Ok(()) } @@ -2329,8 +2517,12 @@ fn parallelization_single_partition() -> Result<()> { " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", " DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; - assert_optimized!(expected_parquet, plan_parquet, true, false, 2, true, 10); - assert_optimized!(expected_csv, plan_csv, true, false, 2, true, 10); + + let test_config = TestConfig::new(DoFirst::Distribution) + .with_prefer_repartition_file_scans(10) + .with_query_execution_partitions(2); + assert_optimized!(expected_parquet, plan_parquet, &test_config); + assert_optimized!(expected_csv, plan_csv, &test_config); Ok(()) } @@ -2346,24 +2538,22 @@ fn parallelization_multiple_files() -> Result<()> { let plan = filter_exec(parquet_exec_multiple_sorted(vec![sort_key.clone()])); let plan = sort_required_exec_with_req(plan, sort_key); + let test_config = TestConfig::new(DoFirst::Distribution) + .with_prefer_existing_sort() + .with_prefer_repartition_file_scans(1); + // The groups must have only contiguous ranges of rows from the same file // if any group has rows from multiple files, the data is no longer sorted destroyed // https://github.com/apache/datafusion/issues/8451 let expected = [ "SortRequiredExec: [a@0 ASC]", " FilterExec: c@2 = 0", - " DataSourceExec: file_groups={3 groups: [[x:0..50], [y:0..100], [x:50..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; - let target_partitions = 3; - let repartition_size = 1; + " DataSourceExec: file_groups={3 groups: [[x:0..50], [y:0..100], [x:50..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", + ]; assert_optimized!( expected, plan, - true, - true, - target_partitions, - true, - repartition_size, - false + &test_config.clone().with_query_execution_partitions(3) ); let expected = [ @@ -2371,17 +2561,10 @@ fn parallelization_multiple_files() -> Result<()> { " FilterExec: c@2 = 0", " DataSourceExec: file_groups={8 groups: [[x:0..25], [y:0..25], [x:25..50], [y:25..50], [x:50..75], [y:50..75], [x:75..100], [y:75..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; - let target_partitions = 8; - let repartition_size = 1; assert_optimized!( expected, plan, - true, - true, - target_partitions, - true, - repartition_size, - false + &test_config.with_query_execution_partitions(8) ); Ok(()) @@ -2432,7 +2615,13 @@ fn parallelization_compressed_csv() -> Result<()> { .build(), vec![("a".to_string(), "a".to_string())], ); - assert_optimized!(expected, plan, true, false, 2, true, 10, false); + assert_optimized!( + expected, + plan, + &TestConfig::new(DoFirst::Distribution) + .with_query_execution_partitions(2) + .with_prefer_repartition_file_scans(10) + ); } Ok(()) } @@ -2457,8 +2646,11 @@ fn parallelization_two_partitions() -> Result<()> { // Plan already has two partitions " DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; - assert_optimized!(expected_parquet, plan_parquet, true, false, 2, true, 10); - assert_optimized!(expected_csv, plan_csv, true, false, 2, true, 10); + let test_config = TestConfig::new(DoFirst::Distribution) + .with_query_execution_partitions(2) + .with_prefer_repartition_file_scans(10); + assert_optimized!(expected_parquet, plan_parquet, &test_config); + assert_optimized!(expected_csv, plan_csv, &test_config); Ok(()) } @@ -2482,8 +2674,11 @@ fn parallelization_two_partitions_into_four() -> Result<()> { // Multiple source files splitted across partitions " DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; - assert_optimized!(expected_parquet, plan_parquet, true, false, 4, true, 10); - assert_optimized!(expected_csv, plan_csv, true, false, 4, true, 10); + let test_config = TestConfig::new(DoFirst::Distribution) + .with_query_execution_partitions(4) + .with_prefer_repartition_file_scans(10); + assert_optimized!(expected_parquet, plan_parquet, &test_config); + assert_optimized!(expected_csv, plan_csv, &test_config); Ok(()) } @@ -2514,8 +2709,16 @@ fn parallelization_sorted_limit() -> Result<()> { // Doesn't parallelize for SortExec without preserve_partitioning " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; - assert_optimized!(expected_parquet, plan_parquet, true); - assert_optimized!(expected_csv, plan_csv, true); + assert_optimized!( + expected_parquet, + plan_parquet, + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!( + expected_csv, + plan_csv, + &TestConfig::new(DoFirst::Distribution) + ); Ok(()) } @@ -2558,8 +2761,16 @@ fn parallelization_limit_with_filter() -> Result<()> { // SortExec doesn't benefit from input partitioning " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; - assert_optimized!(expected_parquet, plan_parquet, true); - assert_optimized!(expected_csv, plan_csv, true); + assert_optimized!( + expected_parquet, + plan_parquet, + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!( + expected_csv, + plan_csv, + &TestConfig::new(DoFirst::Distribution) + ); Ok(()) } @@ -2606,8 +2817,16 @@ fn parallelization_ignores_limit() -> Result<()> { " LocalLimitExec: fetch=100", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; - assert_optimized!(expected_parquet, plan_parquet, true); - assert_optimized!(expected_csv, plan_csv, true); + assert_optimized!( + expected_parquet, + plan_parquet, + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!( + expected_csv, + plan_csv, + &TestConfig::new(DoFirst::Distribution) + ); Ok(()) } @@ -2635,8 +2854,16 @@ fn parallelization_union_inputs() -> Result<()> { " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; - assert_optimized!(expected_parquet, plan_parquet, true); - assert_optimized!(expected_csv, plan_csv, true); + assert_optimized!( + expected_parquet, + plan_parquet, + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!( + expected_csv, + plan_csv, + &TestConfig::new(DoFirst::Distribution) + ); Ok(()) } @@ -2663,8 +2890,16 @@ fn parallelization_prior_to_sort_preserving_merge() -> Result<()> { let expected_csv = &[ "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", ]; - assert_optimized!(expected_parquet, plan_parquet, true); - assert_optimized!(expected_csv, plan_csv, true); + assert_optimized!( + expected_parquet, + plan_parquet, + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!( + expected_csv, + plan_csv, + &TestConfig::new(DoFirst::Distribution) + ); Ok(()) } @@ -2697,8 +2932,16 @@ fn parallelization_sort_preserving_merge_with_union() -> Result<()> { " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", ]; - assert_optimized!(expected_parquet, plan_parquet, true); - assert_optimized!(expected_csv, plan_csv, true); + assert_optimized!( + expected_parquet, + plan_parquet, + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!( + expected_csv, + plan_csv, + &TestConfig::new(DoFirst::Distribution) + ); Ok(()) } @@ -2728,8 +2971,16 @@ fn parallelization_does_not_benefit() -> Result<()> { "SortRequiredExec: [c@2 ASC]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", ]; - assert_optimized!(expected_parquet, plan_parquet, true); - assert_optimized!(expected_csv, plan_csv, true); + assert_optimized!( + expected_parquet, + plan_parquet, + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!( + expected_csv, + plan_csv, + &TestConfig::new(DoFirst::Distribution) + ); Ok(()) } @@ -2768,7 +3019,11 @@ fn parallelization_ignores_transitively_with_projection_parquet() -> Result<()> "ProjectionExec: expr=[a@0 as a2, c@2 as c2]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; - assert_optimized!(expected_parquet, plan_parquet, true); + assert_optimized!( + expected_parquet, + plan_parquet, + &TestConfig::new(DoFirst::Distribution) + ); Ok(()) } @@ -2807,7 +3062,11 @@ fn parallelization_ignores_transitively_with_projection_csv() -> Result<()> { "ProjectionExec: expr=[a@0 as a2, c@2 as c2]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", ]; - assert_optimized!(expected_csv, plan_csv, true); + assert_optimized!( + expected_csv, + plan_csv, + &TestConfig::new(DoFirst::Distribution) + ); Ok(()) } @@ -2831,12 +3090,17 @@ fn remove_redundant_roundrobins() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected, physical_plan.clone(), true); - assert_optimized!(expected, physical_plan, false); + assert_optimized!( + expected, + physical_plan.clone(), + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!(expected, physical_plan, &TestConfig::new(DoFirst::Sorting)); Ok(()) } +/// This test case uses [`TestConfig::with_prefer_existing_sort`]. #[test] fn remove_unnecessary_spm_after_filter() -> Result<()> { let schema = schema(); @@ -2855,13 +3119,21 @@ fn remove_unnecessary_spm_after_filter() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=c@2 ASC", " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; - // last flag sets config.optimizer.PREFER_EXISTING_SORT - assert_optimized!(expected, physical_plan.clone(), true, true); - assert_optimized!(expected, physical_plan, false, true); + assert_optimized!( + expected, + physical_plan.clone(), + &TestConfig::new(DoFirst::Distribution).with_prefer_existing_sort() + ); + assert_optimized!( + expected, + physical_plan, + &TestConfig::new(DoFirst::Sorting).with_prefer_existing_sort() + ); Ok(()) } +/// This test case uses [`TestConfig::with_prefer_existing_sort`]. #[test] fn preserve_ordering_through_repartition() -> Result<()> { let schema = schema(); @@ -2878,9 +3150,16 @@ fn preserve_ordering_through_repartition() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=d@3 ASC", " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet", ]; - // last flag sets config.optimizer.PREFER_EXISTING_SORT - assert_optimized!(expected, physical_plan.clone(), true, true); - assert_optimized!(expected, physical_plan, false, true); + assert_optimized!( + expected, + physical_plan.clone(), + &TestConfig::new(DoFirst::Distribution).with_prefer_existing_sort() + ); + assert_optimized!( + expected, + physical_plan, + &TestConfig::new(DoFirst::Sorting).with_prefer_existing_sort() + ); Ok(()) } @@ -2903,7 +3182,11 @@ fn do_not_preserve_ordering_through_repartition() -> Result<()> { " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; - assert_optimized!(expected, physical_plan.clone(), true); + assert_optimized!( + expected, + physical_plan.clone(), + &TestConfig::new(DoFirst::Distribution) + ); let expected = &[ "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", @@ -2912,7 +3195,7 @@ fn do_not_preserve_ordering_through_repartition() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; - assert_optimized!(expected, physical_plan, false); + assert_optimized!(expected, physical_plan, &TestConfig::new(DoFirst::Sorting)); Ok(()) } @@ -2935,8 +3218,12 @@ fn no_need_for_sort_after_filter() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; - assert_optimized!(expected, physical_plan.clone(), true); - assert_optimized!(expected, physical_plan, false); + assert_optimized!( + expected, + physical_plan.clone(), + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!(expected, physical_plan, &TestConfig::new(DoFirst::Sorting)); Ok(()) } @@ -2964,7 +3251,11 @@ fn do_not_preserve_ordering_through_repartition2() -> Result<()> { " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; - assert_optimized!(expected, physical_plan.clone(), true); + assert_optimized!( + expected, + physical_plan.clone(), + &TestConfig::new(DoFirst::Distribution) + ); let expected = &[ "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", @@ -2974,7 +3265,7 @@ fn do_not_preserve_ordering_through_repartition2() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; - assert_optimized!(expected, physical_plan, false); + assert_optimized!(expected, physical_plan, &TestConfig::new(DoFirst::Sorting)); Ok(()) } @@ -2994,8 +3285,12 @@ fn do_not_preserve_ordering_through_repartition3() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; - assert_optimized!(expected, physical_plan.clone(), true); - assert_optimized!(expected, physical_plan, false); + assert_optimized!( + expected, + physical_plan.clone(), + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!(expected, physical_plan, &TestConfig::new(DoFirst::Sorting)); Ok(()) } @@ -3091,8 +3386,16 @@ fn do_not_add_unnecessary_hash() -> Result<()> { " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; // Make sure target partition number is 1. In this case hash repartition is unnecessary - assert_optimized!(expected, physical_plan.clone(), true, false, 1, false, 1024); - assert_optimized!(expected, physical_plan, false, false, 1, false, 1024); + assert_optimized!( + expected, + physical_plan.clone(), + &TestConfig::new(DoFirst::Distribution).with_query_execution_partitions(1) + ); + assert_optimized!( + expected, + physical_plan, + &TestConfig::new(DoFirst::Sorting).with_query_execution_partitions(1) + ); Ok(()) } @@ -3121,8 +3424,16 @@ fn do_not_add_unnecessary_hash2() -> Result<()> { " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; // Make sure target partition number is larger than 2 (e.g partition number at the source). - assert_optimized!(expected, physical_plan.clone(), true, false, 4, false, 1024); - assert_optimized!(expected, physical_plan, false, false, 4, false, 1024); + assert_optimized!( + expected, + physical_plan.clone(), + &TestConfig::new(DoFirst::Distribution).with_query_execution_partitions(4) + ); + assert_optimized!( + expected, + physical_plan, + &TestConfig::new(DoFirst::Sorting).with_query_execution_partitions(4) + ); Ok(()) } @@ -3139,8 +3450,12 @@ fn optimize_away_unnecessary_repartition() -> Result<()> { let expected = &["DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet"]; - assert_optimized!(expected, physical_plan.clone(), true); - assert_optimized!(expected, physical_plan, false); + assert_optimized!( + expected, + physical_plan.clone(), + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!(expected, physical_plan, &TestConfig::new(DoFirst::Sorting)); Ok(()) } @@ -3166,8 +3481,12 @@ fn optimize_away_unnecessary_repartition2() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected, physical_plan.clone(), true); - assert_optimized!(expected, physical_plan, false); + assert_optimized!( + expected, + physical_plan.clone(), + &TestConfig::new(DoFirst::Distribution) + ); + assert_optimized!(expected, physical_plan, &TestConfig::new(DoFirst::Sorting)); Ok(()) } From 755f9a5def64b1a51e1e5a6f894cc85122b4fc48 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 7 Mar 2025 09:20:37 -0500 Subject: [PATCH 24/33] Update ring to v0.17.13 (#15063) --- Cargo.lock | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 3cad2fd73d50..986d66cbd237 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5061,9 +5061,9 @@ dependencies = [ [[package]] name = "ring" -version = "0.17.9" +version = "0.17.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e75ec5e92c4d8aede845126adc388046234541629e76029599ed35a003c7ed24" +checksum = "70ac5d832aa16abd7d1def883a8545280c20a60f523a370aa3a9617c2b8550ee" dependencies = [ "cc", "cfg-if", From 17e8fa75af36e4911b59c3ef757523dedc026182 Mon Sep 17 00:00:00 2001 From: YuNing Chen Date: Fri, 7 Mar 2025 22:27:31 +0800 Subject: [PATCH 25/33] Remove deprecated function `OptimizerRule::try_optimize` (#15051) * chore: cleanup deprecated optimizer API since version <= 40 follow up of #15027 * chore: inlined `optimize_plan_node` And also removed out dated comment * chore: deprecate `supports_rewrite` function --- datafusion/optimizer/src/lib.rs | 2 - datafusion/optimizer/src/optimizer.rs | 56 ++++--------------- .../simplify_expressions/simplify_exprs.rs | 2 - datafusion/optimizer/src/utils.rs | 40 ------------- 4 files changed, 10 insertions(+), 90 deletions(-) diff --git a/datafusion/optimizer/src/lib.rs b/datafusion/optimizer/src/lib.rs index 1280bf2f466e..ce198560805a 100644 --- a/datafusion/optimizer/src/lib.rs +++ b/datafusion/optimizer/src/lib.rs @@ -69,8 +69,6 @@ pub use analyzer::{Analyzer, AnalyzerRule}; pub use optimizer::{ ApplyOrder, Optimizer, OptimizerConfig, OptimizerContext, OptimizerRule, }; -#[allow(deprecated)] -pub use utils::optimize_children; pub(crate) mod join_key_set; mod plan_signature; diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index 018ad8ace0e3..3a69bd91e749 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -69,24 +69,6 @@ use crate::utils::log_plan; /// [`AnalyzerRule`]: crate::analyzer::AnalyzerRule /// [`SessionState::add_optimizer_rule`]: https://docs.rs/datafusion/latest/datafusion/execution/session_state/struct.SessionState.html#method.add_optimizer_rule pub trait OptimizerRule: Debug { - /// Try and rewrite `plan` to an optimized form, returning None if the plan - /// cannot be optimized by this rule. - /// - /// Note this API will be deprecated in the future as it requires `clone`ing - /// the input plan, which can be expensive. OptimizerRules should implement - /// [`Self::rewrite`] instead. - #[deprecated( - since = "40.0.0", - note = "please implement supports_rewrite and rewrite instead" - )] - fn try_optimize( - &self, - _plan: &LogicalPlan, - _config: &dyn OptimizerConfig, - ) -> Result> { - internal_err!("Should have called rewrite") - } - /// A human readable name for this optimizer rule fn name(&self) -> &str; @@ -99,15 +81,13 @@ pub trait OptimizerRule: Debug { } /// Does this rule support rewriting owned plans (rather than by reference)? + #[deprecated(since = "47.0.0", note = "This method is no longer used")] fn supports_rewrite(&self) -> bool { true } /// Try to rewrite `plan` to an optimized form, returning `Transformed::yes` /// if the plan was rewritten and `Transformed::no` if it was not. - /// - /// Note: this function is only called if [`Self::supports_rewrite`] returns - /// true. Otherwise the Optimizer calls [`Self::try_optimize`] fn rewrite( &self, _plan: LogicalPlan, @@ -304,7 +284,9 @@ impl TreeNodeRewriter for Rewriter<'_> { fn f_down(&mut self, node: LogicalPlan) -> Result> { if self.apply_order == ApplyOrder::TopDown { - optimize_plan_node(node, self.rule, self.config) + { + self.rule.rewrite(node, self.config) + } } else { Ok(Transformed::no(node)) } @@ -312,35 +294,15 @@ impl TreeNodeRewriter for Rewriter<'_> { fn f_up(&mut self, node: LogicalPlan) -> Result> { if self.apply_order == ApplyOrder::BottomUp { - optimize_plan_node(node, self.rule, self.config) + { + self.rule.rewrite(node, self.config) + } } else { Ok(Transformed::no(node)) } } } -/// Invokes the Optimizer rule to rewrite the LogicalPlan in place. -fn optimize_plan_node( - plan: LogicalPlan, - rule: &dyn OptimizerRule, - config: &dyn OptimizerConfig, -) -> Result> { - if rule.supports_rewrite() { - return rule.rewrite(plan, config); - } - - #[allow(deprecated)] - rule.try_optimize(&plan, config).map(|maybe_plan| { - match maybe_plan { - Some(new_plan) => { - // if the node was rewritten by the optimizer, replace the node - Transformed::yes(new_plan) - } - None => Transformed::no(plan), - } - }) -} - impl Optimizer { /// Optimizes the logical plan by applying optimizer rules, and /// invoking observer function after each call @@ -386,7 +348,9 @@ impl Optimizer { &mut Rewriter::new(apply_order, rule.as_ref(), config), ), // rule handles recursion itself - None => optimize_plan_node(new_plan, rule.as_ref(), config), + None => { + rule.rewrite(new_plan, config) + }, } .and_then(|tnr| { // run checks optimizer invariant checks, per optimizer rule applied diff --git a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs index 6a56c1753328..709d8f79c3d9 100644 --- a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs +++ b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs @@ -62,8 +62,6 @@ impl OptimizerRule for SimplifyExpressions { true } - /// if supports_owned returns true, the Optimizer calls - /// [`Self::rewrite`] instead of [`Self::try_optimize`] fn rewrite( &self, plan: LogicalPlan, diff --git a/datafusion/optimizer/src/utils.rs b/datafusion/optimizer/src/utils.rs index 39f8cf285d17..c734d908f6d6 100644 --- a/datafusion/optimizer/src/utils.rs +++ b/datafusion/optimizer/src/utils.rs @@ -19,8 +19,6 @@ use std::collections::{BTreeSet, HashMap, HashSet}; -use crate::{OptimizerConfig, OptimizerRule}; - use crate::analyzer::type_coercion::TypeCoercionRewriter; use arrow::array::{new_null_array, Array, RecordBatch}; use arrow::datatypes::{DataType, Field, Schema}; @@ -38,44 +36,6 @@ use std::sync::Arc; /// as it was initially placed here and then moved elsewhere. pub use datafusion_expr::expr_rewriter::NamePreserver; -/// Convenience rule for writing optimizers: recursively invoke -/// optimize on plan's children and then return a node of the same -/// type. Useful for optimizer rules which want to leave the type -/// of plan unchanged but still apply to the children. -/// This also handles the case when the `plan` is a [`LogicalPlan::Explain`]. -/// -/// Returning `Ok(None)` indicates that the plan can't be optimized by the `optimizer`. -#[deprecated( - since = "40.0.0", - note = "please use OptimizerRule::apply_order with ApplyOrder::BottomUp instead" -)] -pub fn optimize_children( - optimizer: &impl OptimizerRule, - plan: &LogicalPlan, - config: &dyn OptimizerConfig, -) -> Result> { - let mut new_inputs = Vec::with_capacity(plan.inputs().len()); - let mut plan_is_changed = false; - for input in plan.inputs() { - if optimizer.supports_rewrite() { - let new_input = optimizer.rewrite(input.clone(), config)?; - plan_is_changed = plan_is_changed || new_input.transformed; - new_inputs.push(new_input.data); - } else { - #[allow(deprecated)] - let new_input = optimizer.try_optimize(input, config)?; - plan_is_changed = plan_is_changed || new_input.is_some(); - new_inputs.push(new_input.unwrap_or_else(|| input.clone())) - } - } - if plan_is_changed { - let exprs = plan.expressions(); - plan.with_new_exprs(exprs, new_inputs).map(Some) - } else { - Ok(None) - } -} - /// Returns true if `expr` contains all columns in `schema_cols` pub(crate) fn has_all_column_refs(expr: &Expr, schema_cols: &HashSet) -> bool { let column_refs = expr.column_refs(); From 24586938518d747c16dd0702454fb334ab426a85 Mon Sep 17 00:00:00 2001 From: Qi Zhu <821684824@qq.com> Date: Fri, 7 Mar 2025 23:08:42 +0800 Subject: [PATCH 26/33] Minor: fix CI to make the sqllogic testing result consistent (#15059) --- .github/workflows/rust.yml | 4 ++-- datafusion/sqllogictest/Cargo.toml | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index 99aaa7d6f290..f37e5c9f7ef1 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -295,7 +295,7 @@ jobs: export RUST_MIN_STACK=20971520 export TPCH_DATA=`realpath datafusion/sqllogictest/test_files/tpch/data` cargo test plan_q --package datafusion-benchmarks --profile ci --features=ci -- --test-threads=1 - INCLUDE_TPCH=true cargo test --profile ci --package datafusion-sqllogictest --test sqllogictests + INCLUDE_TPCH=true cargo test --features backtrace --profile ci --package datafusion-sqllogictest --test sqllogictests - name: Verify Working Directory Clean run: git diff --exit-code @@ -331,7 +331,7 @@ jobs: - name: Run sqllogictest run: | cd datafusion/sqllogictest - PG_COMPAT=true PG_URI="postgresql://postgres:postgres@$POSTGRES_HOST:$POSTGRES_PORT/db_test" cargo test --profile ci --features=postgres --test sqllogictests + PG_COMPAT=true PG_URI="postgresql://postgres:postgres@$POSTGRES_HOST:$POSTGRES_PORT/db_test" cargo test --features backtrace --profile ci --features=postgres --test sqllogictests env: # use postgres for the host here because we have specified a container for the job POSTGRES_HOST: postgres diff --git a/datafusion/sqllogictest/Cargo.toml b/datafusion/sqllogictest/Cargo.toml index 22223d23e155..33abe77032ca 100644 --- a/datafusion/sqllogictest/Cargo.toml +++ b/datafusion/sqllogictest/Cargo.toml @@ -66,6 +66,7 @@ tokio-postgres = { version = "0.7.12", optional = true } [features] avro = ["datafusion/avro"] +backtrace = ["datafusion/backtrace"] postgres = [ "bytes", "chrono", From 13b731c469d095af24326648e22c05732d79a69b Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" <193874+carols10cents@users.noreply.github.com> Date: Fri, 7 Mar 2025 15:36:25 -0500 Subject: [PATCH 27/33] Improve benchmark documentation (#15054) * Replace placeholder with a more appropriate benchmark title * Clarify purpose of this benchmark * Add cancellation benchmark to benchmarks README --- benchmarks/README.md | 32 +++++++++++++++++++++++++++++++- benchmarks/src/cancellation.rs | 5 ++++- 2 files changed, 35 insertions(+), 2 deletions(-) diff --git a/benchmarks/README.md b/benchmarks/README.md index c0bdb1dba719..40d467b5bb72 100644 --- a/benchmarks/README.md +++ b/benchmarks/README.md @@ -329,7 +329,37 @@ Your benchmark should create and use an instance of `BenchmarkRun` defined in `b # Benchmarks -The output of `dfbench` help includes a description of each benchmark, which is reproduced here for convenience +The output of `dfbench` help includes a description of each benchmark, which is reproduced here for convenience. + +## Cancellation + +Test performance of cancelling queries +Queries in DataFusion should stop executing "quickly" after they are +cancelled (the output stream is dropped). + +The queries are executed on a synthetic dataset generated during +the benchmark execution that is an anonymized version of a +real-world data set. + +The query is an anonymized version of a real-world query, and the +test starts the query then cancels it and reports how long it takes +for the runtime to fully exit. + +Example output: + +``` +Using 7 files found on disk +Starting to load data into in-memory object store +Done loading data into in-memory object store +in main, sleeping +Starting spawned +Creating logical plan... +Creating physical plan... +Executing physical plan... +Getting results... +cancelling thread +done dropping runtime in 83.531417ms +``` ## ClickBench diff --git a/benchmarks/src/cancellation.rs b/benchmarks/src/cancellation.rs index 3c3ca424a308..f5740bdc96e0 100644 --- a/benchmarks/src/cancellation.rs +++ b/benchmarks/src/cancellation.rs @@ -47,6 +47,9 @@ use tokio_util::sync::CancellationToken; /// Test performance of cancelling queries /// +/// Queries in DataFusion should stop executing "quickly" after they are +/// cancelled (the output stream is dropped). +/// /// The queries are executed on a synthetic dataset generated during /// the benchmark execution that is an anonymized version of a /// real-world data set. @@ -97,7 +100,7 @@ impl RunOpt { println!("Done loading data into in-memory object store"); let mut rundata = BenchmarkRun::new(); - rundata.start_new_case("Arglebargle"); + rundata.start_new_case("Cancellation"); for i in 0..self.common.iterations { let elapsed = run_test(self.wait_time, Arc::clone(&store))?; From 450c861a8c38f8934b134368949606aaf3287792 Mon Sep 17 00:00:00 2001 From: wiedld Date: Fri, 7 Mar 2025 14:01:09 -0800 Subject: [PATCH 28/33] Refactor SortPushdown using the standard top-down visitor and using `EquivalenceProperties` (#14821) * refactor: have sort pushdown use transform_down, and provide minor refactor in sort_pushdown_helper to make it more understandable * test: inconsequential single change in test * Use consistent variable naming * chore: update variable naming * refactor: only sync the plan children when required * fix: have orderings include constants which are heterogenius across partitions * Revert "fix: have orderings include constants which are heterogenius across partitions" This reverts commit 47753541bdd1322f610b9ae72e922037d2999a12. * test: temporary commit to demonstrate changes that only occur with no partition by (in window agg), and when aggregating on an unordered column * Revert "test: temporary commit to demonstrate changes that only occur with no partition by (in window agg), and when aggregating on an unordered column" This reverts commit 2ee747f87b1abc2c94f13f793a5b2879e6c37197. * chore: cleanup after merging main, for anticipated test change * chore: rename variable * refactor: added test cases for orthogonal sorting, and remove 1 unneeded conditional * chore: remove unneeded conditional and make a comment --------- Co-authored-by: Andrew Lamb --- .../enforce_distribution.rs | 2 +- .../physical_optimizer/enforce_sorting.rs | 102 ++++++++++- .../tests/physical_optimizer/test_utils.rs | 10 +- .../src/enforce_sorting/sort_pushdown.rs | 162 ++++++++++-------- datafusion/physical-optimizer/src/utils.rs | 4 + 5 files changed, 199 insertions(+), 81 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index f22a896c1894..080a10c7b046 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -2388,7 +2388,7 @@ fn repartition_transitively_past_sort_with_projection() -> Result<()> { ); let expected = &[ - "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", // Since this projection is trivial, increasing parallelism is not beneficial " ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index 26a00ef0f29c..bb77192e05b8 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -21,10 +21,10 @@ use crate::physical_optimizer::test_utils::{ aggregate_exec, bounded_window_exec, check_integrity, coalesce_batches_exec, coalesce_partitions_exec, create_test_schema, create_test_schema2, create_test_schema3, filter_exec, global_limit_exec, hash_join_exec, limit_exec, - local_limit_exec, memory_exec, parquet_exec, repartition_exec, sort_exec, sort_expr, - sort_expr_options, sort_merge_join_exec, sort_preserving_merge_exec, - sort_preserving_merge_exec_with_fetch, spr_repartition_exec, stream_exec_ordered, - union_exec, RequirementsTestExec, + local_limit_exec, memory_exec, parquet_exec, repartition_exec, sort_exec, + sort_exec_with_fetch, sort_expr, sort_expr_options, sort_merge_join_exec, + sort_preserving_merge_exec, sort_preserving_merge_exec_with_fetch, + spr_repartition_exec, stream_exec_ordered, union_exec, RequirementsTestExec, }; use arrow::compute::SortOptions; @@ -2242,7 +2242,7 @@ async fn test_window_partial_constant_and_set_monotonicity() -> Result<()> { " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC NULLS LAST], file_type=parquet", ], expected_plan: vec![ - "SortExec: expr=[non_nullable_col@1 ASC NULLS LAST, count@2 ASC NULLS LAST], preserve_partitioning=[false]", + "SortExec: expr=[non_nullable_col@1 ASC NULLS LAST], preserve_partitioning=[false]", " WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC NULLS LAST], file_type=parquet", ], @@ -2259,7 +2259,7 @@ async fn test_window_partial_constant_and_set_monotonicity() -> Result<()> { " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC NULLS LAST], file_type=parquet", ], expected_plan: vec![ - "SortExec: expr=[non_nullable_col@1 DESC NULLS LAST, max@2 DESC NULLS LAST], preserve_partitioning=[false]", + "SortExec: expr=[non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", " WindowAggExec: wdw=[max: Ok(Field { name: \"max\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC NULLS LAST], file_type=parquet", ], @@ -2276,7 +2276,7 @@ async fn test_window_partial_constant_and_set_monotonicity() -> Result<()> { " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC NULLS LAST], file_type=parquet", ], expected_plan: vec![ - "SortExec: expr=[min@2 ASC NULLS LAST, non_nullable_col@1 ASC NULLS LAST], preserve_partitioning=[false]", + "SortExec: expr=[non_nullable_col@1 ASC NULLS LAST], preserve_partitioning=[false]", " WindowAggExec: wdw=[min: Ok(Field { name: \"min\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC NULLS LAST], file_type=parquet", ], @@ -2293,7 +2293,7 @@ async fn test_window_partial_constant_and_set_monotonicity() -> Result<()> { " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC NULLS LAST], file_type=parquet", ], expected_plan: vec![ - "SortExec: expr=[avg@2 DESC NULLS LAST, nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", + "SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", " WindowAggExec: wdw=[avg: Ok(Field { name: \"avg\", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC NULLS LAST], file_type=parquet", ], @@ -3346,3 +3346,89 @@ async fn test_window_partial_constant_and_set_monotonicity() -> Result<()> { Ok(()) } + +#[test] +fn test_removes_unused_orthogonal_sort() -> Result<()> { + let schema = create_test_schema3()?; + let input_sort_exprs = vec![sort_expr("b", &schema), sort_expr("c", &schema)]; + let unbounded_input = stream_exec_ordered(&schema, input_sort_exprs.clone()); + + let orthogonal_sort = sort_exec(vec![sort_expr("a", &schema)], unbounded_input); + let output_sort = sort_exec(input_sort_exprs, orthogonal_sort); // same sort as data source + + // Test scenario/input has an orthogonal sort: + let expected_input = [ + "SortExec: expr=[b@1 ASC, c@2 ASC], preserve_partitioning=[false]", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[b@1 ASC, c@2 ASC]" + ]; + assert_eq!(get_plan_string(&output_sort), expected_input,); + + // Test: should remove orthogonal sort, and the uppermost (unneeded) sort: + let expected_optimized = [ + "StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[b@1 ASC, c@2 ASC]" + ]; + assert_optimized!(expected_input, expected_optimized, output_sort, true); + + Ok(()) +} + +#[test] +fn test_keeps_used_orthogonal_sort() -> Result<()> { + let schema = create_test_schema3()?; + let input_sort_exprs = vec![sort_expr("b", &schema), sort_expr("c", &schema)]; + let unbounded_input = stream_exec_ordered(&schema, input_sort_exprs.clone()); + + let orthogonal_sort = + sort_exec_with_fetch(vec![sort_expr("a", &schema)], Some(3), unbounded_input); // has fetch, so this orthogonal sort changes the output + let output_sort = sort_exec(input_sort_exprs, orthogonal_sort); + + // Test scenario/input has an orthogonal sort: + let expected_input = [ + "SortExec: expr=[b@1 ASC, c@2 ASC], preserve_partitioning=[false]", + " SortExec: TopK(fetch=3), expr=[a@0 ASC], preserve_partitioning=[false]", + " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[b@1 ASC, c@2 ASC]" + ]; + assert_eq!(get_plan_string(&output_sort), expected_input,); + + // Test: should keep the orthogonal sort, since it modifies the output: + let expected_optimized = expected_input; + assert_optimized!(expected_input, expected_optimized, output_sort, true); + + Ok(()) +} + +#[test] +fn test_handles_multiple_orthogonal_sorts() -> Result<()> { + let schema = create_test_schema3()?; + let input_sort_exprs = vec![sort_expr("b", &schema), sort_expr("c", &schema)]; + let unbounded_input = stream_exec_ordered(&schema, input_sort_exprs.clone()); + + let orthogonal_sort_0 = sort_exec(vec![sort_expr("c", &schema)], unbounded_input); // has no fetch, so can be removed + let orthogonal_sort_1 = + sort_exec_with_fetch(vec![sort_expr("a", &schema)], Some(3), orthogonal_sort_0); // has fetch, so this orthogonal sort changes the output + let orthogonal_sort_2 = sort_exec(vec![sort_expr("c", &schema)], orthogonal_sort_1); // has no fetch, so can be removed + let orthogonal_sort_3 = sort_exec(vec![sort_expr("a", &schema)], orthogonal_sort_2); // has no fetch, so can be removed + let output_sort = sort_exec(input_sort_exprs, orthogonal_sort_3); // final sort + + // Test scenario/input has an orthogonal sort: + let expected_input = [ + "SortExec: expr=[b@1 ASC, c@2 ASC], preserve_partitioning=[false]", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + " SortExec: TopK(fetch=3), expr=[a@0 ASC], preserve_partitioning=[false]", + " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[b@1 ASC, c@2 ASC]", + ]; + assert_eq!(get_plan_string(&output_sort), expected_input,); + + // Test: should keep only the needed orthogonal sort, and remove the unneeded ones: + let expected_optimized = [ + "SortExec: expr=[b@1 ASC, c@2 ASC], preserve_partitioning=[false]", + " SortExec: TopK(fetch=3), expr=[a@0 ASC], preserve_partitioning=[false]", + " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[b@1 ASC, c@2 ASC]", + ]; + assert_optimized!(expected_input, expected_optimized, output_sort, true); + + Ok(()) +} diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index 1b8e754ee357..99a75e6e5067 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -295,9 +295,17 @@ pub fn coalesce_batches_exec(input: Arc) -> Arc, input: Arc, +) -> Arc { + sort_exec_with_fetch(sort_exprs, None, input) +} + +pub fn sort_exec_with_fetch( + sort_exprs: impl IntoIterator, + fetch: Option, + input: Arc, ) -> Arc { let sort_exprs = sort_exprs.into_iter().collect(); - Arc::new(SortExec::new(sort_exprs, input)) + Arc::new(SortExec::new(sort_exprs, input).with_fetch(fetch)) } /// A test [`ExecutionPlan`] whose requirements can be configured. diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index 17acb6272938..2e20608d0e9e 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -23,9 +23,7 @@ use crate::utils::{ }; use arrow::datatypes::SchemaRef; -use datafusion_common::tree_node::{ - ConcreteTreeNode, Transformed, TreeNode, TreeNodeRecursion, -}; +use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{plan_err, HashSet, JoinSide, Result}; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; @@ -59,9 +57,9 @@ pub struct ParentRequirements { pub type SortPushDown = PlanContext; /// Assigns the ordering requirement of the root node to the its children. -pub fn assign_initial_requirements(node: &mut SortPushDown) { - let reqs = node.plan.required_input_ordering(); - for (child, requirement) in node.children.iter_mut().zip(reqs) { +pub fn assign_initial_requirements(sort_push_down: &mut SortPushDown) { + let reqs = sort_push_down.plan.required_input_ordering(); + for (child, requirement) in sort_push_down.children.iter_mut().zip(reqs) { child.data = ParentRequirements { ordering_requirement: requirement, // If the parent has a fetch value, assign it to the children @@ -71,24 +69,26 @@ pub fn assign_initial_requirements(node: &mut SortPushDown) { } } -pub fn pushdown_sorts(sort_pushdown: SortPushDown) -> Result { - let mut new_node = pushdown_sorts_helper(sort_pushdown)?; - while new_node.tnr == TreeNodeRecursion::Stop { - new_node = pushdown_sorts_helper(new_node.data)?; +pub fn pushdown_sorts(sort_push_down: SortPushDown) -> Result { + sort_push_down + .transform_down(pushdown_sorts_helper) + .map(|transformed| transformed.data) +} + +fn min_fetch(f1: Option, f2: Option) -> Option { + match (f1, f2) { + (Some(f1), Some(f2)) => Some(f1.min(f2)), + (Some(_), _) => f1, + (_, Some(_)) => f2, + _ => None, } - let (new_node, children) = new_node.data.take_children(); - let new_children = children - .into_iter() - .map(pushdown_sorts) - .collect::>()?; - new_node.with_new_children(new_children) } fn pushdown_sorts_helper( - mut requirements: SortPushDown, + mut sort_push_down: SortPushDown, ) -> Result> { - let plan = &requirements.plan; - let parent_reqs = requirements + let plan = &sort_push_down.plan; + let parent_reqs = sort_push_down .data .ordering_requirement .clone() @@ -98,82 +98,102 @@ fn pushdown_sorts_helper( .ordering_satisfy_requirement(&parent_reqs); if is_sort(plan) { - let sort_fetch = plan.fetch(); - let required_ordering = plan + let current_sort_fetch = plan.fetch(); + let parent_req_fetch = sort_push_down.data.fetch; + + let current_plan_reqs = plan .output_ordering() .cloned() .map(LexRequirement::from) .unwrap_or_default(); - if !satisfy_parent { - // Make sure this `SortExec` satisfies parent requirements: - let sort_reqs = requirements.data.ordering_requirement.unwrap_or_default(); - // It's possible current plan (`SortExec`) has a fetch value. - // And if both of them have fetch values, we should use the minimum one. - if let Some(fetch) = sort_fetch { - if let Some(requirement_fetch) = requirements.data.fetch { - requirements.data.fetch = Some(fetch.min(requirement_fetch)); - } - } - let fetch = requirements.data.fetch.or(sort_fetch); - requirements = requirements.children.swap_remove(0); - requirements = add_sort_above(requirements, sort_reqs, fetch); - }; + let parent_is_stricter = plan + .equivalence_properties() + .requirements_compatible(&parent_reqs, ¤t_plan_reqs); + let current_is_stricter = plan + .equivalence_properties() + .requirements_compatible(¤t_plan_reqs, &parent_reqs); + + if !satisfy_parent && !parent_is_stricter { + // This new sort has different requirements than the ordering being pushed down. + // 1. add a `SortExec` here for the pushed down ordering (parent reqs). + // 2. continue sort pushdown, but with the new ordering of the new sort. + + // remove current sort (which will be the new ordering to pushdown) + let new_reqs = current_plan_reqs; + sort_push_down = sort_push_down.children.swap_remove(0); + sort_push_down = sort_push_down.update_plan_from_children()?; // changed plan + + // add back sort exec matching parent + sort_push_down = + add_sort_above(sort_push_down, parent_reqs, parent_req_fetch); + + // make pushdown requirements be the new ones. + sort_push_down.children[0].data = ParentRequirements { + ordering_requirement: Some(new_reqs), + fetch: current_sort_fetch, + }; + } else { + // Don't add a SortExec + // Do update what sort requirements to keep pushing down - // We can safely get the 0th index as we are dealing with a `SortExec`. - let mut child = requirements.children.swap_remove(0); - if let Some(adjusted) = - pushdown_requirement_to_children(&child.plan, &required_ordering)? - { - let fetch = sort_fetch.or_else(|| child.plan.fetch()); - for (grand_child, order) in child.children.iter_mut().zip(adjusted) { - grand_child.data = ParentRequirements { - ordering_requirement: order, - fetch, - }; + // remove current sort, and get the sort's child + sort_push_down = sort_push_down.children.swap_remove(0); + sort_push_down = sort_push_down.update_plan_from_children()?; // changed plan + + // set the stricter fetch + sort_push_down.data.fetch = min_fetch(current_sort_fetch, parent_req_fetch); + + // set the stricter ordering + if current_is_stricter { + sort_push_down.data.ordering_requirement = Some(current_plan_reqs); + } else { + sort_push_down.data.ordering_requirement = Some(parent_reqs); } - // Can push down requirements - child.data = ParentRequirements { - ordering_requirement: Some(required_ordering), - fetch, - }; - return Ok(Transformed { - data: child, - transformed: true, - tnr: TreeNodeRecursion::Stop, - }); - } else { - // Can not push down requirements - requirements.children = vec![child]; - assign_initial_requirements(&mut requirements); + // recursive call to helper, so it doesn't transform_down and miss the new node (previous child of sort) + return pushdown_sorts_helper(sort_push_down); } + } else if parent_reqs.is_empty() { + // note: this `satisfy_parent`, but we don't want to push down anything. + // Nothing to do. + return Ok(Transformed::no(sort_push_down)); } else if satisfy_parent { - // For non-sort operators, immediately return if parent requirements are met: + // For non-sort operators which satisfy ordering: let reqs = plan.required_input_ordering(); - for (child, order) in requirements.children.iter_mut().zip(reqs) { + let parent_req_fetch = sort_push_down.data.fetch; + + for (child, order) in sort_push_down.children.iter_mut().zip(reqs) { child.data.ordering_requirement = order; + child.data.fetch = min_fetch(parent_req_fetch, child.data.fetch); } } else if let Some(adjusted) = pushdown_requirement_to_children(plan, &parent_reqs)? { - // Can not satisfy the parent requirements, check whether we can push - // requirements down: - for (child, order) in requirements.children.iter_mut().zip(adjusted) { + // For operators that can take a sort pushdown. + + // Continue pushdown, with updated requirements: + let parent_fetch = sort_push_down.data.fetch; + let current_fetch = plan.fetch(); + for (child, order) in sort_push_down.children.iter_mut().zip(adjusted) { child.data.ordering_requirement = order; + child.data.fetch = min_fetch(current_fetch, parent_fetch); } - requirements.data.ordering_requirement = None; + sort_push_down.data.ordering_requirement = None; } else { // Can not push down requirements, add new `SortExec`: - let sort_reqs = requirements + let sort_reqs = sort_push_down .data .ordering_requirement .clone() .unwrap_or_default(); - let fetch = requirements.data.fetch; - requirements = add_sort_above(requirements, sort_reqs, fetch); - assign_initial_requirements(&mut requirements); + let fetch = sort_push_down.data.fetch; + sort_push_down = add_sort_above(sort_push_down, sort_reqs, fetch); + assign_initial_requirements(&mut sort_push_down); } - Ok(Transformed::yes(requirements)) + + Ok(Transformed::yes(sort_push_down)) } +/// Calculate the pushdown ordering requirements for children. +/// If sort cannot be pushed down, return None. fn pushdown_requirement_to_children( plan: &Arc, parent_required: &LexRequirement, diff --git a/datafusion/physical-optimizer/src/utils.rs b/datafusion/physical-optimizer/src/utils.rs index 636e78a06ce7..57a193315a5c 100644 --- a/datafusion/physical-optimizer/src/utils.rs +++ b/datafusion/physical-optimizer/src/utils.rs @@ -31,6 +31,10 @@ use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties}; /// This utility function adds a `SortExec` above an operator according to the /// given ordering requirements while preserving the original partitioning. +/// +/// Note that this updates the plan in both the [`PlanContext.children`] and +/// the [`PlanContext.plan`]'s children. Therefore its not required to sync +/// the child plans with [`PlanContext::update_plan_from_children`]. pub fn add_sort_above( node: PlanContext, sort_requirements: LexRequirement, From 6952d5ed2cef9fde1aa3f304a36a282f96a43d60 Mon Sep 17 00:00:00 2001 From: irenjj Date: Sat, 8 Mar 2025 06:07:48 +0800 Subject: [PATCH 29/33] Improve explain tree formatting for longer lines / word wrap (#15031) * Improve explain tree formatting for longer lines / word wrap * fix redundant output * fix previous test error output * fix * fix issues * fix test --- datafusion/physical-plan/src/display.rs | 54 +++++++- .../sqllogictest/test_files/explain_tree.slt | 122 +++++++++++++++++- 2 files changed, 173 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-plan/src/display.rs b/datafusion/physical-plan/src/display.rs index 096b51540745..98ba3e1fd9a0 100644 --- a/datafusion/physical-plan/src/display.rs +++ b/datafusion/physical-plan/src/display.rs @@ -890,9 +890,13 @@ impl TreeRenderVisitor<'_, '_> { splits = truncated_splits; } for split in splits { - // TODO: check every line is less than MAX_LINE_RENDER_SIZE. - result.push(split); + Self::split_string_buffer(&split, result); } + if result.len() > max_lines { + result.truncate(max_lines); + result.push("...".to_string()); + } + requires_padding = true; was_inlined = is_inlined; } @@ -945,6 +949,52 @@ impl TreeRenderVisitor<'_, '_> { false } + + fn split_string_buffer(source: &str, result: &mut Vec) { + let mut character_pos = 0; + let mut start_pos = 0; + let mut render_width = 0; + let mut last_possible_split = 0; + + let chars: Vec = source.chars().collect(); + + while character_pos < chars.len() { + // Treating each char as width 1 for simplification + let char_width = 1; + + // Does the next character make us exceed the line length? + if render_width + char_width > Self::NODE_RENDER_WIDTH - 2 { + if start_pos + 8 > last_possible_split { + // The last character we can split on is one of the first 8 characters of the line + // to not create very small lines we instead split on the current character + last_possible_split = character_pos; + } + + result.push(source[start_pos..last_possible_split].to_string()); + render_width = character_pos - last_possible_split; + start_pos = last_possible_split; + character_pos = last_possible_split; + } + + // check if we can split on this character + if Self::can_split_on_this_char(chars[character_pos]) { + last_possible_split = character_pos; + } + + character_pos += 1; + render_width += char_width; + } + + if source.len() > start_pos { + // append the remainder of the input + result.push(source[start_pos..].to_string()); + } + } + + fn can_split_on_this_char(c: char) -> bool { + (!c.is_ascii_digit() && !c.is_ascii_uppercase() && !c.is_ascii_lowercase()) + && c != '_' + } } /// Trait for types which could have additional details when formatted in `Verbose` mode diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index 9d50b9bd626e..9659bdae195d 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -245,7 +245,97 @@ physical_plan 05)│ FilterExec │ 06)│ -------------------- │ 07)│ predicate: │ -08)│string_col@1 != foo AND ...│ +08)│ string_col@1 != foo AND │ +09)│ string_col@1 != bar │ +10)│ AND string_col@1 != a │ +11)│ really long string │ +12)│ constant │ +13)└─────────────┬─────────────┘ +14)┌─────────────┴─────────────┐ +15)│ RepartitionExec │ +16)└─────────────┬─────────────┘ +17)┌─────────────┴─────────────┐ +18)│ DataSourceExec │ +19)│ -------------------- │ +20)│ files: 1 │ +21)│ format: csv │ +22)└───────────────────────────┘ + +# Check maximum line limit. +query TT +explain SELECT int_col FROM table1 +WHERE string_col != 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'; +---- +logical_plan +01)Projection: table1.int_col +02)--Filter: table1.string_col != Utf8("aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa") +03)----TableScan: table1 projection=[int_col, string_col], partial_filters=[table1.string_col != Utf8("aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa")] +physical_plan +01)┌───────────────────────────┐ +02)│ CoalesceBatchesExec │ +03)└─────────────┬─────────────┘ +04)┌─────────────┴─────────────┐ +05)│ FilterExec │ +06)│ -------------------- │ +07)│ predicate: │ +08)│ string_col@1 != │ +09)│ aaaaaaaaaaaaaa │ +10)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +11)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +12)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +13)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +14)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +15)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +16)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +17)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +18)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +19)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +20)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +21)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +22)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +23)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +24)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +25)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +26)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +27)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +28)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +29)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +30)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +31)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +32)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +33)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +34)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +35)│aaaaaaaaaaaaaaaaaaaaaaaaaaa│ +36)│ ... │ +37)└─────────────┬─────────────┘ +38)┌─────────────┴─────────────┐ +39)│ RepartitionExec │ +40)└─────────────┬─────────────┘ +41)┌─────────────┴─────────────┐ +42)│ DataSourceExec │ +43)│ -------------------- │ +44)│ files: 1 │ +45)│ format: csv │ +46)└───────────────────────────┘ + +# Check exactly the render width. +query TT +explain SELECT int_col FROM table1 +WHERE string_col != 'aaaaaaaaaaa'; +---- +logical_plan +01)Projection: table1.int_col +02)--Filter: table1.string_col != Utf8("aaaaaaaaaaa") +03)----TableScan: table1 projection=[int_col, string_col], partial_filters=[table1.string_col != Utf8("aaaaaaaaaaa")] +physical_plan +01)┌───────────────────────────┐ +02)│ CoalesceBatchesExec │ +03)└─────────────┬─────────────┘ +04)┌─────────────┴─────────────┐ +05)│ FilterExec │ +06)│ -------------------- │ +07)│ predicate: │ +08)│string_col@1 != aaaaaaaaaaa│ 09)└─────────────┬─────────────┘ 10)┌─────────────┴─────────────┐ 11)│ RepartitionExec │ @@ -257,6 +347,36 @@ physical_plan 17)│ format: csv │ 18)└───────────────────────────┘ +# Check with the render witdth + 1. +query TT +explain SELECT int_col FROM table1 +WHERE string_col != 'aaaaaaaaaaaa'; +---- +logical_plan +01)Projection: table1.int_col +02)--Filter: table1.string_col != Utf8("aaaaaaaaaaaa") +03)----TableScan: table1 projection=[int_col, string_col], partial_filters=[table1.string_col != Utf8("aaaaaaaaaaaa")] +physical_plan +01)┌───────────────────────────┐ +02)│ CoalesceBatchesExec │ +03)└─────────────┬─────────────┘ +04)┌─────────────┴─────────────┐ +05)│ FilterExec │ +06)│ -------------------- │ +07)│ predicate: │ +08)│ string_col@1 != │ +09)│ aaaaaaaaaaaa │ +10)└─────────────┬─────────────┘ +11)┌─────────────┴─────────────┐ +12)│ RepartitionExec │ +13)└─────────────┬─────────────┘ +14)┌─────────────┴─────────────┐ +15)│ DataSourceExec │ +16)│ -------------------- │ +17)│ files: 1 │ +18)│ format: csv │ +19)└───────────────────────────┘ + # Query with filter on csv query TT explain SELECT int_col FROM table1 WHERE string_col != 'foo'; From f8c9b8f0991c1b96a672b975ea0b9bb6d2047624 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 7 Mar 2025 18:53:22 -0500 Subject: [PATCH 30/33] chore(deps): bump sqllogictest from 0.27.2 to 0.28.0 (#15060) Bumps [sqllogictest](https://github.com/risinglightdb/sqllogictest-rs) from 0.27.2 to 0.28.0. - [Release notes](https://github.com/risinglightdb/sqllogictest-rs/releases) - [Changelog](https://github.com/risinglightdb/sqllogictest-rs/blob/main/CHANGELOG.md) - [Commits](https://github.com/risinglightdb/sqllogictest-rs/compare/v0.27.2...v0.28.0) --- updated-dependencies: - dependency-name: sqllogictest dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Andrew Lamb --- Cargo.lock | 4 ++-- datafusion/sqllogictest/Cargo.toml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 986d66cbd237..a5491d1547dc 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5693,9 +5693,9 @@ dependencies = [ [[package]] name = "sqllogictest" -version = "0.27.2" +version = "0.28.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6f1c93848602f92e5925690d4805ccbc1ccdb61bee7d4ae79ad6862b542a539c" +checksum = "17b2f0b80fc250ed3fdd82fc88c0ada5ad62ee1ed5314ac5474acfa52082f518" dependencies = [ "async-trait", "educe", diff --git a/datafusion/sqllogictest/Cargo.toml b/datafusion/sqllogictest/Cargo.toml index 33abe77032ca..8c3ed1ebc868 100644 --- a/datafusion/sqllogictest/Cargo.toml +++ b/datafusion/sqllogictest/Cargo.toml @@ -55,7 +55,7 @@ postgres-types = { version = "0.2.8", features = ["derive", "with-chrono-0_4"], rust_decimal = { version = "1.36.0", features = ["tokio-pg"] } # When updating the following dependency verify that sqlite test file regeneration works correctly # by running the regenerate_sqlite_files.sh script. -sqllogictest = "0.27.2" +sqllogictest = "0.28.0" sqlparser = { workspace = true } tempfile = { workspace = true } testcontainers = { version = "0.23", features = ["default"], optional = true } From c247b0250499c0d71ad98b89dc69b6d4cc93d832 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 7 Mar 2025 18:53:34 -0500 Subject: [PATCH 31/33] chore(deps): bump async-compression from 0.4.18 to 0.4.19 (#15061) Bumps [async-compression](https://github.com/Nullus157/async-compression) from 0.4.18 to 0.4.19. - [Release notes](https://github.com/Nullus157/async-compression/releases) - [Changelog](https://github.com/Nullus157/async-compression/blob/main/CHANGELOG.md) - [Commits](https://github.com/Nullus157/async-compression/compare/v0.4.18...v0.4.19) --- updated-dependencies: - dependency-name: async-compression dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Andrew Lamb --- Cargo.lock | 6 +++--- datafusion/datasource/Cargo.toml | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index a5491d1547dc..b7b913b69386 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -516,11 +516,11 @@ dependencies = [ [[package]] name = "async-compression" -version = "0.4.18" +version = "0.4.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "df895a515f70646414f4b45c0b79082783b80552b373a68283012928df56f522" +checksum = "06575e6a9673580f52661c92107baabffbf41e2141373441cbcdc47cb733003c" dependencies = [ - "bzip2 0.4.4", + "bzip2 0.5.1", "flate2", "futures-core", "memchr", diff --git a/datafusion/datasource/Cargo.toml b/datafusion/datasource/Cargo.toml index 5f0b2b4f2e77..473800c7779f 100644 --- a/datafusion/datasource/Cargo.toml +++ b/datafusion/datasource/Cargo.toml @@ -37,7 +37,7 @@ default = ["compression"] [dependencies] arrow = { workspace = true } -async-compression = { version = "0.4.0", features = [ +async-compression = { version = "0.4.19", features = [ "bzip2", "gzip", "xz", From 8356c94b83727cc21d1bb1364b39a555ce3794cf Mon Sep 17 00:00:00 2001 From: delamarch3 <68732277+delamarch3@users.noreply.github.com> Date: Sat, 8 Mar 2025 01:20:13 +0000 Subject: [PATCH 32/33] Handle columns in with_new_exprs with a Join (#15055) * handle columns in with_new_exprs with Join * test doesn't return result * take join from result * clippy * make test fallible * accept any pair of expression for new_on in with_new_exprs for Join * use with_capacity --- datafusion/expr/src/logical_plan/plan.rs | 130 ++++++++++++++++++++--- 1 file changed, 118 insertions(+), 12 deletions(-) diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 72b82fc219eb..682342d27b29 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -903,7 +903,7 @@ impl LogicalPlan { let (left, right) = self.only_two_inputs(inputs)?; let schema = build_join_schema(left.schema(), right.schema(), join_type)?; - let equi_expr_count = on.len(); + let equi_expr_count = on.len() * 2; assert!(expr.len() >= equi_expr_count); // Assume that the last expr, if any, @@ -917,17 +917,16 @@ impl LogicalPlan { // The first part of expr is equi-exprs, // and the struct of each equi-expr is like `left-expr = right-expr`. assert_eq!(expr.len(), equi_expr_count); - let new_on = expr.into_iter().map(|equi_expr| { + let mut new_on = Vec::with_capacity(on.len()); + let mut iter = expr.into_iter(); + while let Some(left) = iter.next() { + let Some(right) = iter.next() else { + internal_err!("Expected a pair of expressions to construct the join on expression")? + }; + // SimplifyExpression rule may add alias to the equi_expr. - let unalias_expr = equi_expr.clone().unalias(); - if let Expr::BinaryExpr(BinaryExpr { left, op: Operator::Eq, right }) = unalias_expr { - Ok((*left, *right)) - } else { - internal_err!( - "The front part expressions should be an binary equality expression, actual:{equi_expr}" - ) - } - }).collect::>>()?; + new_on.push((left.unalias(), right.unalias())); + } Ok(LogicalPlan::Join(Join { left: Arc::new(left), @@ -3780,7 +3779,8 @@ mod tests { use crate::builder::LogicalTableSource; use crate::logical_plan::table_scan; use crate::{ - col, exists, in_subquery, lit, placeholder, scalar_subquery, GroupingSet, + binary_expr, col, exists, in_subquery, lit, placeholder, scalar_subquery, + GroupingSet, }; use datafusion_common::tree_node::{ @@ -4632,4 +4632,110 @@ digraph { let parameter_type = params.clone().get(placeholder_value).unwrap().clone(); assert_eq!(parameter_type, None); } + + #[test] + fn test_join_with_new_exprs() -> Result<()> { + fn create_test_join( + on: Vec<(Expr, Expr)>, + filter: Option, + ) -> Result { + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, false), + Field::new("b", DataType::Int32, false), + ]); + + let left_schema = DFSchema::try_from_qualified_schema("t1", &schema)?; + let right_schema = DFSchema::try_from_qualified_schema("t2", &schema)?; + + Ok(LogicalPlan::Join(Join { + left: Arc::new( + table_scan(Some("t1"), left_schema.as_arrow(), None)?.build()?, + ), + right: Arc::new( + table_scan(Some("t2"), right_schema.as_arrow(), None)?.build()?, + ), + on, + filter, + join_type: JoinType::Inner, + join_constraint: JoinConstraint::On, + schema: Arc::new(left_schema.join(&right_schema)?), + null_equals_null: false, + })) + } + + { + let join = create_test_join(vec![(col("t1.a"), (col("t2.a")))], None)?; + let LogicalPlan::Join(join) = join.with_new_exprs( + join.expressions(), + join.inputs().into_iter().cloned().collect(), + )? + else { + unreachable!() + }; + assert_eq!(join.on, vec![(col("t1.a"), (col("t2.a")))]); + assert_eq!(join.filter, None); + } + + { + let join = create_test_join(vec![], Some(col("t1.a").gt(col("t2.a"))))?; + let LogicalPlan::Join(join) = join.with_new_exprs( + join.expressions(), + join.inputs().into_iter().cloned().collect(), + )? + else { + unreachable!() + }; + assert_eq!(join.on, vec![]); + assert_eq!(join.filter, Some(col("t1.a").gt(col("t2.a")))); + } + + { + let join = create_test_join( + vec![(col("t1.a"), (col("t2.a")))], + Some(col("t1.b").gt(col("t2.b"))), + )?; + let LogicalPlan::Join(join) = join.with_new_exprs( + join.expressions(), + join.inputs().into_iter().cloned().collect(), + )? + else { + unreachable!() + }; + assert_eq!(join.on, vec![(col("t1.a"), (col("t2.a")))]); + assert_eq!(join.filter, Some(col("t1.b").gt(col("t2.b")))); + } + + { + let join = create_test_join( + vec![(col("t1.a"), (col("t2.a"))), (col("t1.b"), (col("t2.b")))], + None, + )?; + let LogicalPlan::Join(join) = join.with_new_exprs( + vec![ + binary_expr(col("t1.a"), Operator::Plus, lit(1)), + binary_expr(col("t2.a"), Operator::Plus, lit(2)), + col("t1.b"), + col("t2.b"), + lit(true), + ], + join.inputs().into_iter().cloned().collect(), + )? + else { + unreachable!() + }; + assert_eq!( + join.on, + vec![ + ( + binary_expr(col("t1.a"), Operator::Plus, lit(1)), + binary_expr(col("t2.a"), Operator::Plus, lit(2)) + ), + (col("t1.b"), (col("t2.b"))) + ] + ); + assert_eq!(join.filter, Some(lit(true))); + } + + Ok(()) + } } From b65299729ce217359cbd7ad476143ceff6d6912b Mon Sep 17 00:00:00 2001 From: Danila Baklazhenko Date: Sat, 8 Mar 2025 11:21:15 +0200 Subject: [PATCH 33/33] Add workflow dispatch --- .github/workflows/extended.yml | 37 ++------------ .github/workflows/pr_comment_commands.yml | 60 +++++++++++++++++++++++ 2 files changed, 63 insertions(+), 34 deletions(-) create mode 100644 .github/workflows/pr_comment_commands.yml diff --git a/.github/workflows/extended.yml b/.github/workflows/extended.yml index 3c9dd6f1bc43..b8e05e0b4d9c 100644 --- a/.github/workflows/extended.yml +++ b/.github/workflows/extended.yml @@ -32,36 +32,14 @@ on: push: branches: - main + workflow_dispatch: + - issue_comment: - types: [created] - jobs: - check_trigger: - runs-on: ubuntu-latest - outputs: - should_run: ${{ github.event_name == 'push' || (github.event_name == 'issue_comment' && github.event.issue.pull_request && contains(github.event.comment.body, 'Run extended tests')) }} - # Custom ref is required since issue_comment event points to the default branch rather than pull request branch - ref: ${{ github.event_name == 'push' && github.ref ||format('refs/pull/{0}/head', github.event.issue.number) }} - steps: - - name: Add reaction to comment - if: github.event_name == 'issue_comment' - uses: actions/github-script@v7 - with: - script: | - await github.rest.reactions.createForIssueComment({ - owner: context.repo.owner, - repo: context.repo.repo, - comment_id: context.payload.comment.id, - content: 'rocket' - }); - # Check crate compiles and base cargo check passes linux-build-lib: name: linux build test - needs: [check_trigger] - if: needs.check_trigger.outputs.should_run == 'true' runs-on: ubuntu-latest # note: do not use amd/rust container to preserve disk space steps: @@ -69,7 +47,6 @@ jobs: with: submodules: true fetch-depth: 1 - ref: ${{ needs.check_trigger.outputs.ref }} - name: Install Rust run: | curl --proto '=https' --tlsv1.2 -sSf https://sh.rustup.rs | sh -s -- -y @@ -85,8 +62,7 @@ jobs: # Run extended tests (with feature 'extended_tests') linux-test-extended: name: cargo test 'extended_tests' (amd64) - needs: [linux-build-lib, check_trigger] - if: needs.check_trigger.outputs.should_run == 'true' + needs: [linux-build-lib] runs-on: ubuntu-latest # note: do not use amd/rust container to preserve disk space steps: @@ -94,7 +70,6 @@ jobs: with: submodules: true fetch-depth: 1 - ref: ${{ needs.check_trigger.outputs.ref }} - name: Free Disk Space (Ubuntu) uses: jlumbroso/free-disk-space@54081f138730dfa15788a46383842cd2f914a1be - name: Install Rust @@ -120,8 +95,6 @@ jobs: # Check answers are correct when hash values collide hash-collisions: name: cargo test hash collisions (amd64) - needs: [check_trigger] - if: needs.check_trigger.outputs.should_run == 'true' runs-on: ubuntu-latest container: image: amd64/rust @@ -130,7 +103,6 @@ jobs: with: submodules: true fetch-depth: 1 - ref: ${{ needs.check_trigger.outputs.ref }} - name: Setup Rust toolchain uses: ./.github/actions/setup-builder with: @@ -143,8 +115,6 @@ jobs: sqllogictest-sqlite: name: "Run sqllogictests with the sqlite test suite" - needs: [check_trigger] - if: needs.check_trigger.outputs.should_run == 'true' runs-on: ubuntu-latest container: image: amd64/rust @@ -153,7 +123,6 @@ jobs: with: submodules: true fetch-depth: 1 - ref: ${{ needs.check_trigger.outputs.ref }} - name: Setup Rust toolchain uses: ./.github/actions/setup-builder with: diff --git a/.github/workflows/pr_comment_commands.yml b/.github/workflows/pr_comment_commands.yml new file mode 100644 index 000000000000..9a558956767b --- /dev/null +++ b/.github/workflows/pr_comment_commands.yml @@ -0,0 +1,60 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +name: PR commands + +on: + issue_comment: + types: [created] + +permissions: + contents: read + pull-requests: write + actions: write + +jobs: + + run_extended_tests: + runs-on: ubuntu-latest + if: ${{ github.event_name == 'issue_comment' && github.event.issue.pull_request && contains(github.event.comment.body, 'Run extended tests') }} + outputs: + # Custom ref is required since issue_comment event points to the default branch rather than pull request branch + ref: ${{ format('refs/pull/{0}/head', github.event.issue.number) }} + steps: + + - name: Dispatch extended tests for a PR branch with comment + uses: actions/github-script@v7 + with: + github-token: ${{ secrets.GITHUB_TOKEN }} + script: | + await github.rest.actions.createWorkflowDispatch({ + owner: context.repo.owner, + repo: context.repo.repo, + workflow_id: 'extended.yml', + ref: 'refs/pull/${context.payload.issue.number}/head', + }); + + - name: Add reaction to comment + uses: actions/github-script@v7 + with: + script: | + await github.rest.reactions.createForIssueComment({ + owner: context.repo.owner, + repo: context.repo.repo, + comment_id: context.payload.comment.id, + content: 'rocket' + }); \ No newline at end of file