You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by al...@apache.org on 2023/06/21 17:17:14 UTC
[arrow-datafusion] branch main updated: `derive(Debug)` for `Expr` (#6708)
This is an automated email from the ASF dual-hosted git repository.
alamb pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/arrow-datafusion.git
The following commit(s) were added to refs/heads/main by this push:
new 1b4153f1e2 `derive(Debug)` for `Expr` (#6708)
1b4153f1e2 is described below
commit 1b4153f1e229b1d6308427318b5f68d0ded8f2ab
Author: parkma99 <84...@users.noreply.github.com>
AuthorDate: Thu Jun 22 01:17:08 2023 +0800
`derive(Debug)` for `Expr` (#6708)
* move to display
* fix expr error
* fix some error
* update
* use micro
* update
* fix
* fix error
* fmt
* fix
* update
* update
* fix case when
---
datafusion/core/src/datasource/listing/table.rs | 4 +-
.../src/physical_optimizer/dist_enforcement.rs | 169 ++++++++++----------
.../core/src/physical_optimizer/join_selection.rs | 4 +-
.../src/physical_optimizer/sort_enforcement.rs | 28 ++--
.../core/src/physical_plan/joins/hash_join.rs | 10 +-
.../src/physical_plan/joins/nested_loop_join.rs | 2 +-
.../src/physical_plan/joins/sort_merge_join.rs | 10 +-
.../src/physical_plan/joins/symmetric_hash_join.rs | 22 ++-
datafusion/core/src/physical_plan/mod.rs | 18 +++
.../core/src/physical_plan/repartition/mod.rs | 2 +-
datafusion/core/src/physical_planner.rs | 6 +-
datafusion/core/tests/sql/explain_analyze.rs | 8 +-
datafusion/core/tests/sql/order.rs | 4 +-
.../core/tests/sqllogictests/test_files/insert.slt | 6 +-
.../core/tests/sqllogictests/test_files/joins.slt | 106 ++++++-------
.../tests/sqllogictests/test_files/subquery.slt | 26 +--
.../sqllogictests/test_files/tpch/q1.slt.part | 2 +-
.../sqllogictests/test_files/tpch/q10.slt.part | 20 +--
.../sqllogictests/test_files/tpch/q11.slt.part | 28 ++--
.../sqllogictests/test_files/tpch/q12.slt.part | 8 +-
.../sqllogictests/test_files/tpch/q13.slt.part | 8 +-
.../sqllogictests/test_files/tpch/q14.slt.part | 6 +-
.../sqllogictests/test_files/tpch/q15.slt.part | 14 +-
.../sqllogictests/test_files/tpch/q16.slt.part | 16 +-
.../sqllogictests/test_files/tpch/q17.slt.part | 12 +-
.../sqllogictests/test_files/tpch/q18.slt.part | 18 +--
.../sqllogictests/test_files/tpch/q19.slt.part | 6 +-
.../sqllogictests/test_files/tpch/q2.slt.part | 50 +++---
.../sqllogictests/test_files/tpch/q20.slt.part | 24 +--
.../sqllogictests/test_files/tpch/q21.slt.part | 30 ++--
.../sqllogictests/test_files/tpch/q22.slt.part | 10 +-
.../sqllogictests/test_files/tpch/q3.slt.part | 14 +-
.../sqllogictests/test_files/tpch/q4.slt.part | 8 +-
.../sqllogictests/test_files/tpch/q5.slt.part | 32 ++--
.../sqllogictests/test_files/tpch/q7.slt.part | 32 ++--
.../sqllogictests/test_files/tpch/q8.slt.part | 44 +++---
.../sqllogictests/test_files/tpch/q9.slt.part | 32 ++--
.../core/tests/sqllogictests/test_files/union.slt | 34 ++--
.../core/tests/sqllogictests/test_files/window.slt | 18 +--
datafusion/expr/src/expr.rs | 176 +++++++++------------
datafusion/expr/src/expr_fn.rs | 4 +-
datafusion/expr/src/expr_rewriter/mod.rs | 4 +-
datafusion/expr/src/logical_plan/builder.rs | 4 +-
datafusion/expr/src/logical_plan/plan.rs | 39 +++--
datafusion/expr/src/operator.rs | 24 +--
datafusion/expr/src/utils.rs | 17 +-
datafusion/optimizer/src/analyzer/subquery.rs | 2 +-
datafusion/optimizer/src/analyzer/type_coercion.rs | 4 +-
datafusion/physical-expr/src/scalar_function.rs | 12 +-
datafusion/sql/src/utils.rs | 9 +-
50 files changed, 605 insertions(+), 581 deletions(-)
diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs
index c14127c45e..9beb88144d 100644
--- a/datafusion/core/src/datasource/listing/table.rs
+++ b/datafusion/core/src/datasource/listing/table.rs
@@ -641,12 +641,12 @@ impl ListingTable {
}
else {
Err(DataFusionError::Plan(
- format!("Expected single column references in output_ordering, got {expr:?}")
+ format!("Expected single column references in output_ordering, got {expr}")
))
}
} else {
Err(DataFusionError::Plan(
- format!("Expected Expr::Sort in output_ordering, but got {expr:?}")
+ format!("Expected Expr::Sort in output_ordering, but got {expr}")
))
}
})
diff --git a/datafusion/core/src/physical_optimizer/dist_enforcement.rs b/datafusion/core/src/physical_optimizer/dist_enforcement.rs
index 89107ac989..cb98e69d7a 100644
--- a/datafusion/core/src/physical_optimizer/dist_enforcement.rs
+++ b/datafusion/core/src/physical_optimizer/dist_enforcement.rs
@@ -1230,8 +1230,9 @@ mod tests {
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=[(Column {{ name: \"a\", index: 0 }}, Column {{ name: \"b1\", index: 1 }})]");
+ let join_plan = format!(
+ "HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, b1@1)]"
+ );
match join_type {
JoinType::Inner
@@ -1252,32 +1253,32 @@ mod tests {
&join_type,
);
let top_join_plan =
- format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(Column {{ name: \"a\", index: 0 }}, Column {{ name: \"c\", index: 2 }})]");
+ format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, c@2)]");
let expected = match join_type {
// Should include 3 RepartitionExecs
JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => vec![
top_join_plan.as_str(),
join_plan.as_str(),
- "RepartitionExec: partitioning=Hash([Column { name: \"a\", index: 0 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
- "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 1 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([b1@1], 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]",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
- "RepartitionExec: partitioning=Hash([Column { name: \"c\", index: 2 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
],
// Should include 4 RepartitionExecs
_ => vec![
top_join_plan.as_str(),
- "RepartitionExec: partitioning=Hash([Column { name: \"a\", index: 0 }], 10), input_partitions=10",
+ "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10",
join_plan.as_str(),
- "RepartitionExec: partitioning=Hash([Column { name: \"a\", index: 0 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
- "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 1 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([b1@1], 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]",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
- "RepartitionExec: partitioning=Hash([Column { name: \"c\", index: 2 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
],
};
@@ -1304,9 +1305,9 @@ mod tests {
hash_join_exec(join, parquet_exec(), &top_join_on, &join_type);
let top_join_plan = match join_type {
JoinType::RightSemi | JoinType::RightAnti =>
- format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(Column {{ name: \"b1\", index: 1 }}, Column {{ name: \"c\", index: 2 }})]"),
+ format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(b1@1, c@2)]"),
_ =>
- format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(Column {{ name: \"b1\", index: 6 }}, Column {{ name: \"c\", index: 2 }})]"),
+ format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(b1@6, c@2)]"),
};
let expected = match join_type {
@@ -1315,26 +1316,26 @@ mod tests {
vec![
top_join_plan.as_str(),
join_plan.as_str(),
- "RepartitionExec: partitioning=Hash([Column { name: \"a\", index: 0 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
- "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 1 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([b1@1], 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]",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
- "RepartitionExec: partitioning=Hash([Column { name: \"c\", index: 2 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
],
// Should include 4 RepartitionExecs
_ =>
vec![
top_join_plan.as_str(),
- "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 6 }], 10), input_partitions=10",
+ "RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10",
join_plan.as_str(),
- "RepartitionExec: partitioning=Hash([Column { name: \"a\", index: 0 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
- "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 1 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([b1@1], 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]",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
- "RepartitionExec: partitioning=Hash([Column { name: \"c\", index: 2 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
],
};
@@ -1381,14 +1382,14 @@ mod tests {
// Output partition need to respect the Alias and should not introduce additional RepartitionExec
let expected = &[
- "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: \"a1\", index: 0 }, Column { name: \"c\", index: 2 })]",
+ "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=[(Column { name: \"a\", index: 0 }, Column { name: \"b\", index: 1 })]",
- "RepartitionExec: partitioning=Hash([Column { name: \"a\", index: 0 }], 10), input_partitions=1",
+ "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]",
+ "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
- "RepartitionExec: partitioning=Hash([Column { name: \"b\", index: 1 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
- "RepartitionExec: partitioning=Hash([Column { name: \"c\", index: 2 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
];
assert_optimized!(expected, top_join);
@@ -1403,14 +1404,14 @@ mod tests {
// Output partition need to respect the Alias and should not introduce additional RepartitionExec
let expected = &[
- "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: \"a2\", index: 1 }, Column { name: \"c\", index: 2 })]",
+ "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=[(Column { name: \"a\", index: 0 }, Column { name: \"b\", index: 1 })]",
- "RepartitionExec: partitioning=Hash([Column { name: \"a\", index: 0 }], 10), input_partitions=1",
+ "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]",
+ "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
- "RepartitionExec: partitioning=Hash([Column { name: \"b\", index: 1 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
- "RepartitionExec: partitioning=Hash([Column { name: \"c\", index: 2 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
];
@@ -1452,16 +1453,16 @@ mod tests {
// The Column 'a' has different meaning now after the two Projections
// 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=[(Column { name: \"a\", index: 0 }, Column { name: \"c\", index: 2 })]",
- "RepartitionExec: partitioning=Hash([Column { name: \"a\", index: 0 }], 10), input_partitions=10",
+ "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=[(Column { name: \"a\", index: 0 }, Column { name: \"b\", index: 1 })]",
- "RepartitionExec: partitioning=Hash([Column { name: \"a\", index: 0 }], 10), input_partitions=1",
+ "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]",
+ "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
- "RepartitionExec: partitioning=Hash([Column { name: \"b\", index: 1 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
- "RepartitionExec: partitioning=Hash([Column { name: \"c\", index: 2 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
];
@@ -1491,13 +1492,13 @@ mod tests {
// Only two RepartitionExecs added
let expected = &[
- "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: \"a1\", index: 0 }, Column { name: \"a2\", index: 0 })]",
+ "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a1@0, a2@0)]",
"AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]",
- "RepartitionExec: partitioning=Hash([Column { name: \"a1\", index: 0 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=1",
"AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
"AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[]",
- "RepartitionExec: partitioning=Hash([Column { name: \"a2\", index: 0 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([a2@0], 10), input_partitions=1",
"AggregateExec: mode=Partial, gby=[a@0 as a2], aggr=[]",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
];
@@ -1539,14 +1540,14 @@ mod tests {
// Only two RepartitionExecs added
let expected = &[
- "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: \"b1\", index: 1 }, Column { name: \"b\", index: 0 }), (Column { name: \"a1\", index: 0 }, Column { name: \"a\", index: 1 })]",
+ "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([Column { name: \"b1\", index: 0 }, Column { name: \"a1\", index: 1 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=1",
"AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
"AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]",
- "RepartitionExec: partitioning=Hash([Column { name: \"b\", index: 0 }, Column { name: \"a\", index: 1 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=1",
"AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
];
@@ -1646,18 +1647,18 @@ mod tests {
// 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=[(Column { name: \"B\", index: 2 }, Column { name: \"b1\", index: 6 }), (Column { name: \"C\", index: 3 }, Column { name: \"c\", index: 2 }), (Column { name: \"AA\", index: 1 }, Column { name: \"a1\", index: 5 })]",
+ "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=[(Column { name: \"b\", index: 1 }, Column { name: \"b1\", index: 1 }), (Column { name: \"c\", index: 2 }, Column { name: \"c1\", index: 2 }), (Column { name: \"a\", index: 0 }, Column { name: \"a1\", index: 0 })]",
- "RepartitionExec: partitioning=Hash([Column { name: \"b\", index: 1 }, Column { name: \"c\", index: 2 }, Column { name: \"a\", index: 0 }], 10), input_partitions=1",
+ "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=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
- "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 1 }, Column { name: \"c1\", index: 2 }, Column { name: \"a1\", index: 0 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=1",
"ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
- "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: \"b\", index: 1 }, Column { name: \"b1\", index: 1 }), (Column { name: \"c\", index: 2 }, Column { name: \"c1\", index: 2 }), (Column { name: \"a\", index: 0 }, Column { name: \"a1\", index: 0 })]",
- "RepartitionExec: partitioning=Hash([Column { name: \"b\", index: 1 }, Column { name: \"c\", index: 2 }, Column { name: \"a\", index: 0 }], 10), input_partitions=1",
+ "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=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
- "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 1 }, Column { name: \"c1\", index: 2 }, Column { name: \"a1\", index: 0 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=1",
"ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
];
@@ -1762,7 +1763,7 @@ mod tests {
&join_type,
);
let top_join_plan =
- format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(Column {{ name: \"AA\", index: 1 }}, Column {{ name: \"a1\", index: 5 }}), (Column {{ name: \"B\", index: 2 }}, Column {{ name: \"b1\", index: 6 }}), (Column {{ name: \"C\", index: 3 }}, Column {{ name: \"c\", index: 2 }})]", &join_type);
+ format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(AA@1, a1@5), (B@2, b1@6), (C@3, c@2)]", &join_type);
let reordered = reorder_join_keys_to_inputs(top_join)?;
@@ -1770,16 +1771,16 @@ mod tests {
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=[(Column { name: \"a\", index: 0 }, Column { name: \"a1\", index: 0 }), (Column { name: \"b\", index: 1 }, Column { name: \"b1\", index: 1 }), (Column { name: \"c\", index: 2 }, Column { name: \"c1\", index: 2 })]",
- "RepartitionExec: partitioning=Hash([Column { name: \"a\", index: 0 }, Column { name: \"b\", index: 1 }, Column { name: \"c\", index: 2 }], 10), input_partitions=1",
+ "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=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
- "RepartitionExec: partitioning=Hash([Column { name: \"a1\", index: 0 }, Column { name: \"b1\", index: 1 }, Column { name: \"c1\", index: 2 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([a1@0, b1@1, c1@2], 10), input_partitions=1",
"ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
- "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: \"c\", index: 2 }, Column { name: \"c1\", index: 2 }), (Column { name: \"b\", index: 1 }, Column { name: \"b1\", index: 1 }), (Column { name: \"a\", index: 0 }, Column { name: \"a1\", index: 0 })]",
- "RepartitionExec: partitioning=Hash([Column { name: \"c\", index: 2 }, Column { name: \"b\", index: 1 }, Column { name: \"a\", index: 0 }], 10), input_partitions=1",
+ "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=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
- "RepartitionExec: partitioning=Hash([Column { name: \"c1\", index: 2 }, Column { name: \"b1\", index: 1 }, Column { name: \"a1\", index: 0 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=1",
"ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
];
@@ -1883,7 +1884,7 @@ mod tests {
&join_type,
);
let top_join_plan =
- format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(Column {{ name: \"C\", index: 3 }}, Column {{ name: \"c\", index: 2 }}), (Column {{ name: \"B\", index: 2 }}, Column {{ name: \"b1\", index: 6 }}), (Column {{ name: \"AA\", index: 1 }}, Column {{ name: \"a1\", index: 5 }})]", &join_type);
+ format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(C@3, c@2), (B@2, b1@6), (AA@1, a1@5)]", &join_type);
let reordered = reorder_join_keys_to_inputs(top_join)?;
@@ -1891,16 +1892,16 @@ mod tests {
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=[(Column { name: \"a\", index: 0 }, Column { name: \"a1\", index: 0 }), (Column { name: \"b\", index: 1 }, Column { name: \"b1\", index: 1 })]",
- "RepartitionExec: partitioning=Hash([Column { name: \"a\", index: 0 }, Column { name: \"b\", index: 1 }], 10), input_partitions=1",
+ "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=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
- "RepartitionExec: partitioning=Hash([Column { name: \"a1\", index: 0 }, Column { name: \"b1\", index: 1 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([a1@0, b1@1], 10), input_partitions=1",
"ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
- "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: \"c\", index: 2 }, Column { name: \"c1\", index: 2 }), (Column { name: \"b\", index: 1 }, Column { name: \"b1\", index: 1 }), (Column { name: \"a\", index: 0 }, Column { name: \"a1\", index: 0 })]",
- "RepartitionExec: partitioning=Hash([Column { name: \"c\", index: 2 }, Column { name: \"b\", index: 1 }, Column { name: \"a\", index: 0 }], 10), input_partitions=1",
+ "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=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
- "RepartitionExec: partitioning=Hash([Column { name: \"c1\", index: 2 }, Column { name: \"b1\", index: 1 }, Column { name: \"a1\", index: 0 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=1",
"ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
];
@@ -1943,7 +1944,7 @@ mod tests {
let join =
sort_merge_join_exec(left.clone(), right.clone(), &join_on, &join_type);
let join_plan =
- format!("SortMergeJoin: join_type={join_type}, on=[(Column {{ name: \"a\", index: 0 }}, Column {{ name: \"b1\", index: 1 }})]");
+ format!("SortMergeJoin: join_type={join_type}, on=[(a@0, b1@1)]");
// Top join on (a == c)
let top_join_on = vec![(
@@ -1957,7 +1958,7 @@ mod tests {
&join_type,
);
let top_join_plan =
- format!("SortMergeJoin: join_type={join_type}, on=[(Column {{ name: \"a\", index: 0 }}, Column {{ name: \"c\", index: 2 }})]");
+ format!("SortMergeJoin: join_type={join_type}, on=[(a@0, c@2)]");
let expected = match join_type {
// Should include 3 RepartitionExecs 3 SortExecs
@@ -1966,31 +1967,31 @@ mod tests {
top_join_plan.as_str(),
join_plan.as_str(),
"SortExec: expr=[a@0 ASC]",
- "RepartitionExec: partitioning=Hash([Column { name: \"a\", index: 0 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
"SortExec: expr=[b1@1 ASC]",
- "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 1 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([b1@1], 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]",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
"SortExec: expr=[c@2 ASC]",
- "RepartitionExec: partitioning=Hash([Column { name: \"c\", index: 2 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
],
// Should include 4 RepartitionExecs
_ => vec![
top_join_plan.as_str(),
"SortExec: expr=[a@0 ASC]",
- "RepartitionExec: partitioning=Hash([Column { name: \"a\", index: 0 }], 10), input_partitions=10",
+ "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10",
join_plan.as_str(),
"SortExec: expr=[a@0 ASC]",
- "RepartitionExec: partitioning=Hash([Column { name: \"a\", index: 0 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
"SortExec: expr=[b1@1 ASC]",
- "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 1 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([b1@1], 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]",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
"SortExec: expr=[c@2 ASC]",
- "RepartitionExec: partitioning=Hash([Column { name: \"c\", index: 2 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
],
};
@@ -2011,7 +2012,7 @@ mod tests {
&join_type,
);
let top_join_plan =
- format!("SortMergeJoin: join_type={join_type}, on=[(Column {{ name: \"b1\", index: 6 }}, Column {{ name: \"c\", index: 2 }})]");
+ format!("SortMergeJoin: join_type={join_type}, on=[(b1@6, c@2)]");
let expected = match join_type {
// Should include 3 RepartitionExecs and 3 SortExecs
@@ -2019,31 +2020,31 @@ mod tests {
top_join_plan.as_str(),
join_plan.as_str(),
"SortExec: expr=[a@0 ASC]",
- "RepartitionExec: partitioning=Hash([Column { name: \"a\", index: 0 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
"SortExec: expr=[b1@1 ASC]",
- "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 1 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([b1@1], 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]",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
"SortExec: expr=[c@2 ASC]",
- "RepartitionExec: partitioning=Hash([Column { name: \"c\", index: 2 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
],
// Should include 4 RepartitionExecs and 4 SortExecs
_ => vec![
top_join_plan.as_str(),
"SortExec: expr=[b1@6 ASC]",
- "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 6 }], 10), input_partitions=10",
+ "RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10",
join_plan.as_str(),
"SortExec: expr=[a@0 ASC]",
- "RepartitionExec: partitioning=Hash([Column { name: \"a\", index: 0 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
"SortExec: expr=[b1@1 ASC]",
- "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 1 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([b1@1], 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]",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
"SortExec: expr=[c@2 ASC]",
- "RepartitionExec: partitioning=Hash([Column { name: \"c\", index: 2 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
],
};
@@ -2104,18 +2105,18 @@ mod tests {
// Only two RepartitionExecs added
let expected = &[
- "SortMergeJoin: join_type=Inner, on=[(Column { name: \"b3\", index: 1 }, Column { name: \"b2\", index: 1 }), (Column { name: \"a3\", index: 0 }, Column { name: \"a2\", index: 0 })]",
+ "SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)]",
"SortExec: expr=[b3@1 ASC,a3@0 ASC]",
"ProjectionExec: expr=[a1@0 as a3, b1@1 as b3]",
"ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]",
"AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[]",
- "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 0 }, Column { name: \"a1\", index: 1 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=1",
"AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
"SortExec: expr=[b2@1 ASC,a2@0 ASC]",
"ProjectionExec: expr=[a@1 as a2, b@0 as b2]",
"AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]",
- "RepartitionExec: partitioning=Hash([Column { name: \"b\", index: 0 }, Column { name: \"a\", index: 1 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=1",
"AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
];
@@ -2178,11 +2179,11 @@ mod tests {
"AggregateExec: mode=Partial, gby=[a1@0 as a2], aggr=[]",
"InterleaveExec",
"AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]",
- "RepartitionExec: partitioning=Hash([Column { name: \"a1\", index: 0 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=1",
"AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
"AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]",
- "RepartitionExec: partitioning=Hash([Column { name: \"a1\", index: 0 }], 10), input_partitions=1",
+ "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=1",
"AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]",
"ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]",
];
diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs
index accf2d5643..a9dec73c36 100644
--- a/datafusion/core/src/physical_optimizer/join_selection.rs
+++ b/datafusion/core/src/physical_optimizer/join_selection.rs
@@ -687,9 +687,9 @@ mod tests {
// has an exact cardinality of 10_000 rows).
let expected = [
"ProjectionExec: expr=[medium_col@2 as medium_col, big_col@0 as big_col, small_col@1 as small_col]",
- " HashJoinExec: mode=CollectLeft, join_type=Right, on=[(Column { name: \"small_col\", index: 1 }, Column { name: \"medium_col\", index: 0 })]",
+ " HashJoinExec: mode=CollectLeft, join_type=Right, on=[(small_col@1, medium_col@0)]",
" ProjectionExec: expr=[big_col@1 as big_col, small_col@0 as small_col]",
- " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(Column { name: \"small_col\", index: 0 }, Column { name: \"big_col\", index: 0 })]",
+ " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(small_col@0, big_col@0)]",
" StatisticsExec: col_count=1, row_count=Some(1000)",
" StatisticsExec: col_count=1, row_count=Some(100000)",
" StatisticsExec: col_count=1, row_count=Some(10000)",
diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs
index 03df84fa57..719c152841 100644
--- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs
+++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs
@@ -2381,10 +2381,12 @@ mod tests {
];
let physical_plan = sort_preserving_merge_exec(sort_exprs.clone(), join);
- let join_plan =
- format!("SortMergeJoin: join_type={join_type}, on=[(Column {{ name: \"nullable_col\", index: 0 }}, Column {{ name: \"col_a\", index: 0 }})]");
- let join_plan2 =
- format!(" SortMergeJoin: join_type={join_type}, on=[(Column {{ name: \"nullable_col\", index: 0 }}, Column {{ name: \"col_a\", index: 0 }})]");
+ let join_plan = format!(
+ "SortMergeJoin: join_type={join_type}, on=[(nullable_col@0, col_a@0)]"
+ );
+ let join_plan2 = format!(
+ " SortMergeJoin: join_type={join_type}, on=[(nullable_col@0, col_a@0)]"
+ );
let expected_input = vec![
"SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]",
join_plan2.as_str(),
@@ -2452,16 +2454,18 @@ mod tests {
];
let physical_plan = sort_preserving_merge_exec(sort_exprs, join);
- let join_plan =
- format!("SortMergeJoin: join_type={join_type}, on=[(Column {{ name: \"nullable_col\", index: 0 }}, Column {{ name: \"col_a\", index: 0 }})]");
+ let join_plan = format!(
+ "SortMergeJoin: join_type={join_type}, on=[(nullable_col@0, col_a@0)]"
+ );
let spm_plan = match join_type {
JoinType::RightAnti => {
"SortPreservingMergeExec: [col_a@0 ASC,col_b@1 ASC]"
}
_ => "SortPreservingMergeExec: [col_a@2 ASC,col_b@3 ASC]",
};
- let join_plan2 =
- format!(" SortMergeJoin: join_type={join_type}, on=[(Column {{ name: \"nullable_col\", index: 0 }}, Column {{ name: \"col_a\", index: 0 }})]");
+ let join_plan2 = format!(
+ " SortMergeJoin: join_type={join_type}, on=[(nullable_col@0, col_a@0)]"
+ );
let expected_input = vec![
spm_plan,
join_plan2.as_str(),
@@ -2521,7 +2525,7 @@ mod tests {
let expected_input = vec![
"SortPreservingMergeExec: [col_b@3 ASC,col_a@2 ASC]",
- " SortMergeJoin: join_type=Inner, on=[(Column { name: \"nullable_col\", index: 0 }, Column { name: \"col_a\", index: 0 })]",
+ " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]",
" ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]",
" ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]",
];
@@ -2529,7 +2533,7 @@ mod tests {
// can not push down the sort requirements, need to add SortExec
let expected_optimized = vec![
"SortExec: expr=[col_b@3 ASC,col_a@2 ASC]",
- " SortMergeJoin: join_type=Inner, on=[(Column { name: \"nullable_col\", index: 0 }, Column { name: \"col_a\", index: 0 })]",
+ " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]",
" SortExec: expr=[nullable_col@0 ASC]",
" ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]",
" SortExec: expr=[col_a@0 ASC]",
@@ -2547,7 +2551,7 @@ mod tests {
let expected_input = vec![
"SortPreservingMergeExec: [nullable_col@0 ASC,col_b@3 ASC,col_a@2 ASC]",
- " SortMergeJoin: join_type=Inner, on=[(Column { name: \"nullable_col\", index: 0 }, Column { name: \"col_a\", index: 0 })]",
+ " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]",
" ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]",
" ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]",
];
@@ -2555,7 +2559,7 @@ mod tests {
// can not push down the sort requirements, need to add SortExec
let expected_optimized = vec![
"SortExec: expr=[nullable_col@0 ASC,col_b@3 ASC,col_a@2 ASC]",
- " SortMergeJoin: join_type=Inner, on=[(Column { name: \"nullable_col\", index: 0 }, Column { name: \"col_a\", index: 0 })]",
+ " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]",
" SortExec: expr=[nullable_col@0 ASC]",
" ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]",
" SortExec: expr=[col_a@0 ASC]",
diff --git a/datafusion/core/src/physical_plan/joins/hash_join.rs b/datafusion/core/src/physical_plan/joins/hash_join.rs
index c89edea4d6..a3c553c9b3 100644
--- a/datafusion/core/src/physical_plan/joins/hash_join.rs
+++ b/datafusion/core/src/physical_plan/joins/hash_join.rs
@@ -427,10 +427,16 @@ impl ExecutionPlan for HashJoinExec {
|| "".to_string(),
|f| format!(", filter={}", f.expression()),
);
+ let on = self
+ .on
+ .iter()
+ .map(|(c1, c2)| format!("({}, {})", c1, c2))
+ .collect::<Vec<String>>()
+ .join(", ");
write!(
f,
- "HashJoinExec: mode={:?}, join_type={:?}, on={:?}{}",
- self.mode, self.join_type, self.on, display_filter
+ "HashJoinExec: mode={:?}, join_type={:?}, on=[{}]{}",
+ self.mode, self.join_type, on, display_filter
)
}
}
diff --git a/datafusion/core/src/physical_plan/joins/nested_loop_join.rs b/datafusion/core/src/physical_plan/joins/nested_loop_join.rs
index bb8c190222..6586456fd2 100644
--- a/datafusion/core/src/physical_plan/joins/nested_loop_join.rs
+++ b/datafusion/core/src/physical_plan/joins/nested_loop_join.rs
@@ -254,7 +254,7 @@ impl ExecutionPlan for NestedLoopJoinExec {
DisplayFormatType::Default | DisplayFormatType::Verbose => {
let display_filter = self.filter.as_ref().map_or_else(
|| "".to_string(),
- |f| format!(", filter={:?}", f.expression()),
+ |f| format!(", filter={}", f.expression()),
);
write!(
f,
diff --git a/datafusion/core/src/physical_plan/joins/sort_merge_join.rs b/datafusion/core/src/physical_plan/joins/sort_merge_join.rs
index d87023093f..bc8c686670 100644
--- a/datafusion/core/src/physical_plan/joins/sort_merge_join.rs
+++ b/datafusion/core/src/physical_plan/joins/sort_merge_join.rs
@@ -364,10 +364,16 @@ impl ExecutionPlan for SortMergeJoinExec {
fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result {
match t {
DisplayFormatType::Default | DisplayFormatType::Verbose => {
+ let on = self
+ .on
+ .iter()
+ .map(|(c1, c2)| format!("({}, {})", c1, c2))
+ .collect::<Vec<String>>()
+ .join(", ");
write!(
f,
- "SortMergeJoin: join_type={:?}, on={:?}",
- self.join_type, self.on
+ "SortMergeJoin: join_type={:?}, on=[{}]",
+ self.join_type, on
)
}
}
diff --git a/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs b/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs
index 490f18ccf4..b46aba2fb5 100644
--- a/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs
+++ b/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs
@@ -467,10 +467,16 @@ impl ExecutionPlan for SymmetricHashJoinExec {
|| "".to_string(),
|f| format!(", filter={}", f.expression()),
);
+ let on = self
+ .on
+ .iter()
+ .map(|(c1, c2)| format!("({}, {})", c1, c2))
+ .collect::<Vec<String>>()
+ .join(", ");
write!(
f,
- "SymmetricHashJoinExec: join_type={:?}, on={:?}{}",
- self.join_type, self.on, display_filter
+ "SymmetricHashJoinExec: join_type={:?}, on=[{}]{}",
+ self.join_type, on, display_filter
)
}
}
@@ -2627,12 +2633,12 @@ mod tests {
let formatted = displayable(physical_plan.as_ref()).indent(true).to_string();
let expected = {
[
- "SymmetricHashJoinExec: join_type=Full, on=[(Column { name: \"a2\", index: 1 }, Column { name: \"a2\", index: 1 })], filter=CAST(a1@0 AS Int64) > CAST(a1@1 AS Int64) + 3 AND CAST(a1@0 AS Int64) < CAST(a1@1 AS Int64) + 10",
+ "SymmetricHashJoinExec: join_type=Full, on=[(a2@1, a2@1)], filter=CAST(a1@0 AS Int64) > CAST(a1@1 AS Int64) + 3 AND CAST(a1@0 AS Int64) < CAST(a1@1 AS Int64) + 10",
" CoalesceBatchesExec: target_batch_size=8192",
- " RepartitionExec: partitioning=Hash([Column { name: \"a2\", index: 1 }], 8), input_partitions=1",
+ " RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=1",
// " CsvExec: file_groups={1 group: [[tempdir/left.csv]]}, projection=[a1, a2], has_header=false",
" CoalesceBatchesExec: target_batch_size=8192",
- " RepartitionExec: partitioning=Hash([Column { name: \"a2\", index: 1 }], 8), input_partitions=1",
+ " RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=1",
// " CsvExec: file_groups={1 group: [[tempdir/right.csv]]}, projection=[a1, a2], has_header=false"
]
};
@@ -2680,12 +2686,12 @@ mod tests {
let formatted = displayable(physical_plan.as_ref()).indent(true).to_string();
let expected = {
[
- "SymmetricHashJoinExec: join_type=Full, on=[(Column { name: \"a2\", index: 1 }, Column { name: \"a2\", index: 1 })], filter=CAST(a1@0 AS Int64) > CAST(a1@1 AS Int64) + 3 AND CAST(a1@0 AS Int64) < CAST(a1@1 AS Int64) + 10",
+ "SymmetricHashJoinExec: join_type=Full, on=[(a2@1, a2@1)], filter=CAST(a1@0 AS Int64) > CAST(a1@1 AS Int64) + 3 AND CAST(a1@0 AS Int64) < CAST(a1@1 AS Int64) + 10",
" CoalesceBatchesExec: target_batch_size=8192",
- " RepartitionExec: partitioning=Hash([Column { name: \"a2\", index: 1 }], 8), input_partitions=1",
+ " RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=1",
// " CsvExec: file_groups={1 group: [[tempdir/left.csv]]}, projection=[a1, a2], has_header=false",
" CoalesceBatchesExec: target_batch_size=8192",
- " RepartitionExec: partitioning=Hash([Column { name: \"a2\", index: 1 }], 8), input_partitions=1",
+ " RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=1",
// " CsvExec: file_groups={1 group: [[tempdir/right.csv]]}, projection=[a1, a2], has_header=false"
]
};
diff --git a/datafusion/core/src/physical_plan/mod.rs b/datafusion/core/src/physical_plan/mod.rs
index e3e9fec247..5abecf6b16 100644
--- a/datafusion/core/src/physical_plan/mod.rs
+++ b/datafusion/core/src/physical_plan/mod.rs
@@ -486,6 +486,24 @@ pub enum Partitioning {
UnknownPartitioning(usize),
}
+impl fmt::Display for Partitioning {
+ fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
+ match self {
+ Partitioning::RoundRobinBatch(size) => write!(f, "RoundRobinBatch({size})"),
+ Partitioning::Hash(phy_exprs, size) => {
+ let phy_exprs_str = phy_exprs
+ .iter()
+ .map(|e| format!("{e}"))
+ .collect::<Vec<String>>()
+ .join(", ");
+ write!(f, "Hash([{phy_exprs_str}], {size})")
+ }
+ Partitioning::UnknownPartitioning(size) => {
+ write!(f, "UnknownPartitioning({size})")
+ }
+ }
+ }
+}
impl Partitioning {
/// Returns the number of partitions in this partitioning scheme
pub fn partition_count(&self) -> usize {
diff --git a/datafusion/core/src/physical_plan/repartition/mod.rs b/datafusion/core/src/physical_plan/repartition/mod.rs
index e18ba302e0..0dc16eaf1d 100644
--- a/datafusion/core/src/physical_plan/repartition/mod.rs
+++ b/datafusion/core/src/physical_plan/repartition/mod.rs
@@ -453,7 +453,7 @@ impl ExecutionPlan for RepartitionExec {
DisplayFormatType::Default | DisplayFormatType::Verbose => {
write!(
f,
- "RepartitionExec: partitioning={:?}, input_partitions={}",
+ "RepartitionExec: partitioning={}, input_partitions={}",
self.partitioning,
self.input.output_partitioning().partition_count()
)
diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs
index d695bcd99a..aa0cc52daf 100644
--- a/datafusion/core/src/physical_planner.rs
+++ b/datafusion/core/src/physical_planner.rs
@@ -122,13 +122,13 @@ fn create_physical_name(e: &Expr, is_first_expr: bool) -> Result<String> {
Expr::Case(case) => {
let mut name = "CASE ".to_string();
if let Some(e) = &case.expr {
- let _ = write!(name, "{e:?} ");
+ let _ = write!(name, "{e} ");
}
for (w, t) in &case.when_then_expr {
- let _ = write!(name, "WHEN {w:?} THEN {t:?} ");
+ let _ = write!(name, "WHEN {w} THEN {t} ");
}
if let Some(e) = &case.else_expr {
- let _ = write!(name, "ELSE {e:?} ");
+ let _ = write!(name, "ELSE {e} ");
}
name += "END";
Ok(name)
diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs
index 974721144f..01bdb629ee 100644
--- a/datafusion/core/tests/sql/explain_analyze.rs
+++ b/datafusion/core/tests/sql/explain_analyze.rs
@@ -604,7 +604,7 @@ async fn test_physical_plan_display_indent() {
" ProjectionExec: expr=[c1@0 as c1, MAX(aggregate_test_100.c12)@1 as MAX(aggregate_test_100.c12), MIN(aggregate_test_100.c12)@2 as the_min]",
" AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[MAX(aggregate_test_100.c12), MIN(aggregate_test_100.c12)]",
" CoalesceBatchesExec: target_batch_size=4096",
- " RepartitionExec: partitioning=Hash([Column { name: \"c1\", index: 0 }], 9000), input_partitions=9000",
+ " RepartitionExec: partitioning=Hash([c1@0], 9000), input_partitions=9000",
" AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[MAX(aggregate_test_100.c12), MIN(aggregate_test_100.c12)]",
" CoalesceBatchesExec: target_batch_size=4096",
" FilterExec: c12@1 < 10",
@@ -646,13 +646,13 @@ async fn test_physical_plan_display_indent_multi_children() {
let expected = vec![
"ProjectionExec: expr=[c1@0 as c1]",
" CoalesceBatchesExec: target_batch_size=4096",
- " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: \"c1\", index: 0 }, Column { name: \"c2\", index: 0 })]",
+ " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c1@0, c2@0)]",
" CoalesceBatchesExec: target_batch_size=4096",
- " RepartitionExec: partitioning=Hash([Column { name: \"c1\", index: 0 }], 9000), input_partitions=9000",
+ " RepartitionExec: partitioning=Hash([c1@0], 9000), input_partitions=9000",
" RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1",
" CsvExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true",
" CoalesceBatchesExec: target_batch_size=4096",
- " RepartitionExec: partitioning=Hash([Column { name: \"c2\", index: 0 }], 9000), input_partitions=9000",
+ " RepartitionExec: partitioning=Hash([c2@0], 9000), input_partitions=9000",
" RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1",
" ProjectionExec: expr=[c1@0 as c2]",
" CsvExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true",
diff --git a/datafusion/core/tests/sql/order.rs b/datafusion/core/tests/sql/order.rs
index 2054a9937c..3981fbaa4d 100644
--- a/datafusion/core/tests/sql/order.rs
+++ b/datafusion/core/tests/sql/order.rs
@@ -206,7 +206,7 @@ ORDER BY 1, 2;
" ProjectionExec: expr=[Int64(0)@0 as m, t@1 as t]",
" AggregateExec: mode=FinalPartitioned, gby=[Int64(0)@0 as Int64(0), t@1 as t], aggr=[]",
" CoalesceBatchesExec: target_batch_size=8192",
- " RepartitionExec: partitioning=Hash([Column { name: \"Int64(0)\", index: 0 }, Column { name: \"t\", index: 1 }], 2), input_partitions=2",
+ " RepartitionExec: partitioning=Hash([Int64(0)@0, t@1], 2), input_partitions=2",
" AggregateExec: mode=Partial, gby=[0 as Int64(0), t@0 as t], aggr=[]",
" RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1",
" ProjectionExec: expr=[column1@0 as t]",
@@ -214,7 +214,7 @@ ORDER BY 1, 2;
" ProjectionExec: expr=[Int64(1)@0 as m, t@1 as t]",
" AggregateExec: mode=FinalPartitioned, gby=[Int64(1)@0 as Int64(1), t@1 as t], aggr=[]",
" CoalesceBatchesExec: target_batch_size=8192",
- " RepartitionExec: partitioning=Hash([Column { name: \"Int64(1)\", index: 0 }, Column { name: \"t\", index: 1 }], 2), input_partitions=2",
+ " RepartitionExec: partitioning=Hash([Int64(1)@0, t@1], 2), input_partitions=2",
" AggregateExec: mode=Partial, gby=[1 as Int64(1), t@0 as t], aggr=[]",
" RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1",
" ProjectionExec: expr=[column1@0 as t]",
diff --git a/datafusion/core/tests/sqllogictests/test_files/insert.slt b/datafusion/core/tests/sqllogictests/test_files/insert.slt
index ba5b59c45c..2a04eaed32 100644
--- a/datafusion/core/tests/sqllogictests/test_files/insert.slt
+++ b/datafusion/core/tests/sqllogictests/test_files/insert.slt
@@ -71,7 +71,7 @@ InsertExec: sink=MemoryTable (partitions=1)
--------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4): Ok(Field { name: "SUM(aggregate_test_100.c4)", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }, COUNT(UInt8(1)): Ok(Field { name: "COUNT(UInt8(1))", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Prec [...]
----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST]
------------CoalesceBatchesExec: target_batch_size=8192
---------------RepartitionExec: partitioning=Hash([Column { name: "c1", index: 0 }], 8), input_partitions=8
+--------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8
----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1
------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], has_header=true
@@ -131,7 +131,7 @@ InsertExec: sink=MemoryTable (partitions=1)
------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4): Ok(Field { name: "SUM(aggregate_test_100.c4)", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }, COUNT(UInt8(1)): Ok(Field { name: "COUNT(UInt8(1))", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preced [...]
--------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST]
----------CoalesceBatchesExec: target_batch_size=8192
-------------RepartitionExec: partitioning=Hash([Column { name: "c1", index: 0 }], 8), input_partitions=8
+------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8
--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1
----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], has_header=true
@@ -182,7 +182,7 @@ InsertExec: sink=MemoryTable (partitions=8)
--------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4): Ok(Field { name: "SUM(aggregate_test_100.c4)", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }, COUNT(UInt8(1)): Ok(Field { name: "COUNT(UInt8(1))", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Prec [...]
----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST]
------------CoalesceBatchesExec: target_batch_size=8192
---------------RepartitionExec: partitioning=Hash([Column { name: "c1", index: 0 }], 8), input_partitions=8
+--------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8
----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1
------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], has_header=true
diff --git a/datafusion/core/tests/sqllogictests/test_files/joins.slt b/datafusion/core/tests/sqllogictests/test_files/joins.slt
index 72de12b036..95f8999aee 100644
--- a/datafusion/core/tests/sqllogictests/test_files/joins.slt
+++ b/datafusion/core/tests/sqllogictests/test_files/joins.slt
@@ -1312,13 +1312,13 @@ physical_plan
AggregateExec: mode=Single, gby=[t1_id@0 as t1_id], aggr=[]
--ProjectionExec: expr=[t1_id@0 as t1_id]
----CoalesceBatchesExec: target_batch_size=4096
-------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "t1_id", index: 0 }, Column { name: "t2_id", index: 0 })]
+------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)]
--------CoalesceBatchesExec: target_batch_size=4096
-----------RepartitionExec: partitioning=Hash([Column { name: "t1_id", index: 0 }], 2), input_partitions=2
+----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2
------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
--------------MemoryExec: partitions=1, partition_sizes=[1]
--------CoalesceBatchesExec: target_batch_size=4096
-----------RepartitionExec: partitioning=Hash([Column { name: "t2_id", index: 0 }], 2), input_partitions=2
+----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2
------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
--------------MemoryExec: partitions=1, partition_sizes=[1]
@@ -1340,13 +1340,13 @@ ProjectionExec: expr=[COUNT(UInt8(1))@1 as COUNT(UInt8(1))]
--AggregateExec: mode=Single, gby=[t1_id@0 as t1_id], aggr=[COUNT(UInt8(1))]
----ProjectionExec: expr=[t1_id@0 as t1_id]
------CoalesceBatchesExec: target_batch_size=4096
---------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "t1_id", index: 0 }, Column { name: "t2_id", index: 0 })]
+--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)]
----------CoalesceBatchesExec: target_batch_size=4096
-------------RepartitionExec: partitioning=Hash([Column { name: "t1_id", index: 0 }], 2), input_partitions=2
+------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2
--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
----------------MemoryExec: partitions=1, partition_sizes=[1]
----------CoalesceBatchesExec: target_batch_size=4096
-------------RepartitionExec: partitioning=Hash([Column { name: "t2_id", index: 0 }], 2), input_partitions=2
+------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2
--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
----------------MemoryExec: partitions=1, partition_sizes=[1]
@@ -1373,13 +1373,13 @@ ProjectionExec: expr=[COUNT(alias1)@0 as COUNT(DISTINCT join_t1.t1_id)]
----------AggregateExec: mode=Partial, gby=[t1_id@0 as alias1], aggr=[]
------------ProjectionExec: expr=[t1_id@0 as t1_id]
--------------CoalesceBatchesExec: target_batch_size=4096
-----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "t1_id", index: 0 }, Column { name: "t2_id", index: 0 })]
+----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)]
------------------CoalesceBatchesExec: target_batch_size=4096
---------------------RepartitionExec: partitioning=Hash([Column { name: "t1_id", index: 0 }], 2), input_partitions=2
+--------------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2
----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
------------------------MemoryExec: partitions=1, partition_sizes=[1]
------------------CoalesceBatchesExec: target_batch_size=4096
---------------------RepartitionExec: partitioning=Hash([Column { name: "t2_id", index: 0 }], 2), input_partitions=2
+--------------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2
----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
------------------------MemoryExec: partitions=1, partition_sizes=[1]
@@ -1442,7 +1442,7 @@ physical_plan
ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)]
--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@4 as t2_id, t2_name@5 as t2_name, t2_int@6 as t2_int]
----CoalesceBatchesExec: target_batch_size=4096
-------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(Column { name: "join_t1.t1_id + Int64(11)", index: 3 }, Column { name: "CAST(join_t2.t2_id AS Int64)", index: 3 })]
+------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)]
--------CoalescePartitionsExec
----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)]
------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
@@ -1469,14 +1469,14 @@ physical_plan
ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)]
--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@4 as t2_id, t2_name@5 as t2_name, t2_int@6 as t2_int]
----CoalesceBatchesExec: target_batch_size=4096
-------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "join_t1.t1_id + Int64(11)", index: 3 }, Column { name: "CAST(join_t2.t2_id AS Int64)", index: 3 })]
+------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)]
--------CoalesceBatchesExec: target_batch_size=4096
-----------RepartitionExec: partitioning=Hash([Column { name: "join_t1.t1_id + Int64(11)", index: 3 }], 2), input_partitions=2
+----------RepartitionExec: partitioning=Hash([join_t1.t1_id + Int64(11)@3], 2), input_partitions=2
------------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)]
--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
----------------MemoryExec: partitions=1, partition_sizes=[1]
--------CoalesceBatchesExec: target_batch_size=4096
-----------RepartitionExec: partitioning=Hash([Column { name: "CAST(join_t2.t2_id AS Int64)", index: 3 }], 2), input_partitions=2
+----------RepartitionExec: partitioning=Hash([CAST(join_t2.t2_id AS Int64)@3], 2), input_partitions=2
------------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)]
--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
----------------MemoryExec: partitions=1, partition_sizes=[1]
@@ -1501,7 +1501,7 @@ physical_plan
ProjectionExec: expr=[t1_id@0 as t1_id, t2_id@2 as t2_id, t1_name@1 as t1_name]
--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t2_id@3 as t2_id]
----CoalesceBatchesExec: target_batch_size=4096
-------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(Column { name: "join_t1.t1_id + UInt32(12)", index: 2 }, Column { name: "join_t2.t2_id + UInt32(1)", index: 1 })]
+------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + UInt32(12)@2, join_t2.t2_id + UInt32(1)@1)]
--------CoalescePartitionsExec
----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)]
------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
@@ -1528,14 +1528,14 @@ physical_plan
ProjectionExec: expr=[t1_id@0 as t1_id, t2_id@2 as t2_id, t1_name@1 as t1_name]
--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t2_id@3 as t2_id]
----CoalesceBatchesExec: target_batch_size=4096
-------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "join_t1.t1_id + UInt32(12)", index: 2 }, Column { name: "join_t2.t2_id + UInt32(1)", index: 1 })]
+------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t1.t1_id + UInt32(12)@2, join_t2.t2_id + UInt32(1)@1)]
--------CoalesceBatchesExec: target_batch_size=4096
-----------RepartitionExec: partitioning=Hash([Column { name: "join_t1.t1_id + UInt32(12)", index: 2 }], 2), input_partitions=2
+----------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(12)@2], 2), input_partitions=2
------------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)]
--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
----------------MemoryExec: partitions=1, partition_sizes=[1]
--------CoalesceBatchesExec: target_batch_size=4096
-----------RepartitionExec: partitioning=Hash([Column { name: "join_t2.t2_id + UInt32(1)", index: 1 }], 2), input_partitions=2
+----------RepartitionExec: partitioning=Hash([join_t2.t2_id + UInt32(1)@1], 2), input_partitions=2
------------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)]
--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
----------------MemoryExec: partitions=1, partition_sizes=[1]
@@ -1561,7 +1561,7 @@ physical_plan
ProjectionExec: expr=[t1_id@0 as t1_id, t2_id@2 as t2_id, t1_name@1 as t1_name]
--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t2_id@3 as t2_id]
----CoalesceBatchesExec: target_batch_size=4096
-------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(Column { name: "join_t1.t1_id + UInt32(11)", index: 2 }, Column { name: "t2_id", index: 0 })]
+------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + UInt32(11)@2, t2_id@0)]
--------CoalescePartitionsExec
----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)]
------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
@@ -1587,14 +1587,14 @@ physical_plan
ProjectionExec: expr=[t1_id@0 as t1_id, t2_id@2 as t2_id, t1_name@1 as t1_name]
--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t2_id@3 as t2_id]
----CoalesceBatchesExec: target_batch_size=4096
-------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "join_t1.t1_id + UInt32(11)", index: 2 }, Column { name: "t2_id", index: 0 })]
+------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t1.t1_id + UInt32(11)@2, t2_id@0)]
--------CoalesceBatchesExec: target_batch_size=4096
-----------RepartitionExec: partitioning=Hash([Column { name: "join_t1.t1_id + UInt32(11)", index: 2 }], 2), input_partitions=2
+----------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(11)@2], 2), input_partitions=2
------------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)]
--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
----------------MemoryExec: partitions=1, partition_sizes=[1]
--------CoalesceBatchesExec: target_batch_size=4096
-----------RepartitionExec: partitioning=Hash([Column { name: "t2_id", index: 0 }], 2), input_partitions=2
+----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2
------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
--------------MemoryExec: partitions=1, partition_sizes=[1]
@@ -1619,7 +1619,7 @@ physical_plan
ProjectionExec: expr=[t1_id@0 as t1_id, t2_id@2 as t2_id, t1_name@1 as t1_name]
--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t2_id@2 as t2_id]
----CoalesceBatchesExec: target_batch_size=4096
-------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(Column { name: "t1_id", index: 0 }, Column { name: "join_t2.t2_id - UInt32(11)", index: 1 })]
+------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@1)]
--------MemoryExec: partitions=1, partition_sizes=[1]
--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)]
----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
@@ -1644,13 +1644,13 @@ physical_plan
ProjectionExec: expr=[t1_id@0 as t1_id, t2_id@2 as t2_id, t1_name@1 as t1_name]
--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t2_id@2 as t2_id]
----CoalesceBatchesExec: target_batch_size=4096
-------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "t1_id", index: 0 }, Column { name: "join_t2.t2_id - UInt32(11)", index: 1 })]
+------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@1)]
--------CoalesceBatchesExec: target_batch_size=4096
-----------RepartitionExec: partitioning=Hash([Column { name: "t1_id", index: 0 }], 2), input_partitions=2
+----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2
------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
--------------MemoryExec: partitions=1, partition_sizes=[1]
--------CoalesceBatchesExec: target_batch_size=4096
-----------RepartitionExec: partitioning=Hash([Column { name: "join_t2.t2_id - UInt32(11)", index: 1 }], 2), input_partitions=2
+----------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@1], 2), input_partitions=2
------------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)]
--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
----------------MemoryExec: partitions=1, partition_sizes=[1]
@@ -1674,7 +1674,7 @@ Inner Join: join_t1.t1_id = join_t2.t2_id - UInt32(11)
physical_plan
ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int]
--CoalesceBatchesExec: target_batch_size=4096
-----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(Column { name: "t1_id", index: 0 }, Column { name: "join_t2.t2_id - UInt32(11)", index: 3 })]
+----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)]
------MemoryExec: partitions=1, partition_sizes=[1]
------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)]
--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
@@ -1697,13 +1697,13 @@ Inner Join: join_t1.t1_id = join_t2.t2_id - UInt32(11)
physical_plan
ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int]
--CoalesceBatchesExec: target_batch_size=4096
-----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "t1_id", index: 0 }, Column { name: "join_t2.t2_id - UInt32(11)", index: 3 })]
+----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)]
------CoalesceBatchesExec: target_batch_size=4096
---------RepartitionExec: partitioning=Hash([Column { name: "t1_id", index: 0 }], 2), input_partitions=2
+--------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2
----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
------------MemoryExec: partitions=1, partition_sizes=[1]
------CoalesceBatchesExec: target_batch_size=4096
---------RepartitionExec: partitioning=Hash([Column { name: "join_t2.t2_id - UInt32(11)", index: 3 }], 2), input_partitions=2
+--------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@3], 2), input_partitions=2
----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)]
------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
--------------MemoryExec: partitions=1, partition_sizes=[1]
@@ -2024,7 +2024,7 @@ Inner Join: Filter: join_t1.t1_id > join_t2.t2_id
----Filter: join_t2.t2_int > UInt32(1)
------TableScan: join_t2 projection=[t2_id, t2_int]
physical_plan
-NestedLoopJoinExec: join_type=Inner, filter=BinaryExpr { left: Column { name: "t1_id", index: 0 }, op: Gt, right: Column { name: "t2_id", index: 1 } }
+NestedLoopJoinExec: join_type=Inner, filter=t1_id@0 > t2_id@1
--CoalesceBatchesExec: target_batch_size=4096
----FilterExec: t1_id@0 > 10
------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
@@ -2064,7 +2064,7 @@ Right Join: Filter: join_t1.t1_id < join_t2.t2_id
----Filter: join_t2.t2_id > UInt32(11)
------TableScan: join_t2 projection=[t2_id]
physical_plan
-NestedLoopJoinExec: join_type=Right, filter=BinaryExpr { left: Column { name: "t1_id", index: 0 }, op: Lt, right: Column { name: "t2_id", index: 1 } }
+NestedLoopJoinExec: join_type=Right, filter=t1_id@0 < t2_id@1
--CoalescePartitionsExec
----CoalesceBatchesExec: target_batch_size=4096
------FilterExec: t1_id@0 > 22
@@ -2649,15 +2649,15 @@ Inner Join: t1.c1 = t2.c1
--SubqueryAlias: t2
----TableScan: hashjoin_datatype_table_t2 projection=[c1, c2, c3, c4]
physical_plan
-SortMergeJoin: join_type=Inner, on=[(Column { name: "c1", index: 0 }, Column { name: "c1", index: 0 })]
+SortMergeJoin: join_type=Inner, on=[(c1@0, c1@0)]
--SortExec: expr=[c1@0 ASC]
----CoalesceBatchesExec: target_batch_size=4096
-------RepartitionExec: partitioning=Hash([Column { name: "c1", index: 0 }], 2), input_partitions=2
+------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2
--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
----------MemoryExec: partitions=1, partition_sizes=[1]
--SortExec: expr=[c1@0 ASC]
----CoalesceBatchesExec: target_batch_size=4096
-------RepartitionExec: partitioning=Hash([Column { name: "c1", index: 0 }], 2), input_partitions=2
+------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2
--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
----------MemoryExec: partitions=1, partition_sizes=[1]
@@ -2680,16 +2680,16 @@ Right Join: CAST(t1.c3 AS Decimal128(10, 2)) = t2.c3
----TableScan: hashjoin_datatype_table_t2 projection=[c1, c2, c3, c4]
physical_plan
ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, c1@5 as c1, c2@6 as c2, c3@7 as c3, c4@8 as c4]
---SortMergeJoin: join_type=Right, on=[(Column { name: "CAST(t1.c3 AS Decimal128(10, 2))", index: 4 }, Column { name: "c3", index: 2 })]
+--SortMergeJoin: join_type=Right, on=[(CAST(t1.c3 AS Decimal128(10, 2))@4, c3@2)]
----SortExec: expr=[CAST(t1.c3 AS Decimal128(10, 2))@4 ASC]
------CoalesceBatchesExec: target_batch_size=4096
---------RepartitionExec: partitioning=Hash([Column { name: "CAST(t1.c3 AS Decimal128(10, 2))", index: 4 }], 2), input_partitions=2
+--------RepartitionExec: partitioning=Hash([CAST(t1.c3 AS Decimal128(10, 2))@4], 2), input_partitions=2
----------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, CAST(c3@2 AS Decimal128(10, 2)) as CAST(t1.c3 AS Decimal128(10, 2))]
------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
--------------MemoryExec: partitions=1, partition_sizes=[1]
----SortExec: expr=[c3@2 ASC]
------CoalesceBatchesExec: target_batch_size=4096
---------RepartitionExec: partitioning=Hash([Column { name: "c3", index: 2 }], 2), input_partitions=2
+--------RepartitionExec: partitioning=Hash([c3@2], 2), input_partitions=2
----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
------------MemoryExec: partitions=1, partition_sizes=[1]
@@ -2745,13 +2745,13 @@ physical_plan
SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]
--SortExec: expr=[t1_id@0 ASC NULLS LAST]
----CoalesceBatchesExec: target_batch_size=4096
-------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: "t1_id", index: 0 }, Column { name: "t2_id", index: 0 })]
+------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(t1_id@0, t2_id@0)]
--------CoalesceBatchesExec: target_batch_size=4096
-----------RepartitionExec: partitioning=Hash([Column { name: "t1_id", index: 0 }], 2), input_partitions=2
+----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2
------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
--------------MemoryExec: partitions=1, partition_sizes=[1]
--------CoalesceBatchesExec: target_batch_size=4096
-----------RepartitionExec: partitioning=Hash([Column { name: "t2_id", index: 0 }], 2), input_partitions=2
+----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2
------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
--------------MemoryExec: partitions=1, partition_sizes=[1]
@@ -2786,13 +2786,13 @@ physical_plan
SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]
--SortExec: expr=[t1_id@0 ASC NULLS LAST]
----CoalesceBatchesExec: target_batch_size=4096
-------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: "t1_id", index: 0 }, Column { name: "t2_id", index: 0 })]
+------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(t1_id@0, t2_id@0)]
--------CoalesceBatchesExec: target_batch_size=4096
-----------RepartitionExec: partitioning=Hash([Column { name: "t1_id", index: 0 }], 2), input_partitions=2
+----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2
------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
--------------MemoryExec: partitions=1, partition_sizes=[1]
--------CoalesceBatchesExec: target_batch_size=4096
-----------RepartitionExec: partitioning=Hash([Column { name: "t2_id", index: 0 }], 2), input_partitions=2
+----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2
------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
--------------MemoryExec: partitions=1, partition_sizes=[1]
@@ -2847,7 +2847,7 @@ explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 WHERE t1_id I
physical_plan
SortExec: expr=[t1_id@0 ASC NULLS LAST]
--CoalesceBatchesExec: target_batch_size=4096
-----HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(Column { name: "t1_id", index: 0 }, Column { name: "t2_id", index: 0 })]
+----HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(t1_id@0, t2_id@0)]
------MemoryExec: partitions=1, partition_sizes=[1]
------MemoryExec: partitions=1, partition_sizes=[1]
@@ -2881,7 +2881,7 @@ explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 LEFT SEMI JOI
physical_plan
SortExec: expr=[t1_id@0 ASC NULLS LAST]
--CoalesceBatchesExec: target_batch_size=4096
-----HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(Column { name: "t1_id", index: 0 }, Column { name: "t2_id", index: 0 })]
+----HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(t1_id@0, t2_id@0)]
------MemoryExec: partitions=1, partition_sizes=[1]
------MemoryExec: partitions=1, partition_sizes=[1]
@@ -2938,13 +2938,13 @@ physical_plan
SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]
--SortExec: expr=[t1_id@0 ASC NULLS LAST]
----CoalesceBatchesExec: target_batch_size=4096
-------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(Column { name: "t2_id", index: 0 }, Column { name: "t1_id", index: 0 })], filter=t2_name@1 != t1_name@0
+------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0
--------CoalesceBatchesExec: target_batch_size=4096
-----------RepartitionExec: partitioning=Hash([Column { name: "t2_id", index: 0 }], 2), input_partitions=2
+----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2
------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
--------------MemoryExec: partitions=1, partition_sizes=[1]
--------CoalesceBatchesExec: target_batch_size=4096
-----------RepartitionExec: partitioning=Hash([Column { name: "t1_id", index: 0 }], 2), input_partitions=2
+----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2
------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
--------------MemoryExec: partitions=1, partition_sizes=[1]
@@ -2960,13 +2960,13 @@ physical_plan
SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]
--SortExec: expr=[t1_id@0 ASC NULLS LAST]
----CoalesceBatchesExec: target_batch_size=4096
-------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(Column { name: "t2_id", index: 0 }, Column { name: "t1_id", index: 0 })], filter=t2_name@0 != t1_name@1
+------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1
--------CoalesceBatchesExec: target_batch_size=4096
-----------RepartitionExec: partitioning=Hash([Column { name: "t2_id", index: 0 }], 2), input_partitions=2
+----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2
------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
--------------MemoryExec: partitions=1, partition_sizes=[1]
--------CoalesceBatchesExec: target_batch_size=4096
-----------RepartitionExec: partitioning=Hash([Column { name: "t1_id", index: 0 }], 2), input_partitions=2
+----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2
------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
--------------MemoryExec: partitions=1, partition_sizes=[1]
@@ -3019,7 +3019,7 @@ explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t1 t1 WHER
physical_plan
SortExec: expr=[t1_id@0 ASC NULLS LAST]
--CoalesceBatchesExec: target_batch_size=4096
-----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(Column { name: "t2_id", index: 0 }, Column { name: "t1_id", index: 0 })], filter=t2_name@1 != t1_name@0
+----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0
------MemoryExec: partitions=1, partition_sizes=[1]
------MemoryExec: partitions=1, partition_sizes=[1]
@@ -3034,7 +3034,7 @@ explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t2 t2 RIGH
physical_plan
SortExec: expr=[t1_id@0 ASC NULLS LAST]
--CoalesceBatchesExec: target_batch_size=4096
-----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(Column { name: "t2_id", index: 0 }, Column { name: "t1_id", index: 0 })], filter=t2_name@0 != t1_name@1
+----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1
------MemoryExec: partitions=1, partition_sizes=[1]
------MemoryExec: partitions=1, partition_sizes=[1]
diff --git a/datafusion/core/tests/sqllogictests/test_files/subquery.slt b/datafusion/core/tests/sqllogictests/test_files/subquery.slt
index b668891e99..279b019de3 100644
--- a/datafusion/core/tests/sqllogictests/test_files/subquery.slt
+++ b/datafusion/core/tests/sqllogictests/test_files/subquery.slt
@@ -178,14 +178,14 @@ Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int) AS t2_sum
physical_plan
ProjectionExec: expr=[t1_id@0 as t1_id, SUM(t2.t2_int)@1 as t2_sum]
--CoalesceBatchesExec: target_batch_size=8192
-----HashJoinExec: mode=Partitioned, join_type=Left, on=[(Column { name: "t1_id", index: 0 }, Column { name: "t2_id", index: 1 })]
+----HashJoinExec: mode=Partitioned, join_type=Left, on=[(t1_id@0, t2_id@1)]
------CoalesceBatchesExec: target_batch_size=8192
---------RepartitionExec: partitioning=Hash([Column { name: "t1_id", index: 0 }], 4), input_partitions=4
+--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4
----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0]
------ProjectionExec: expr=[SUM(t2.t2_int)@1 as SUM(t2.t2_int), t2_id@0 as t2_id]
--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)]
----------CoalesceBatchesExec: target_batch_size=8192
-------------RepartitionExec: partitioning=Hash([Column { name: "t2_id", index: 0 }], 4), input_partitions=4
+------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4
--------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)]
----------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0]
@@ -212,14 +212,14 @@ Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int * Float64(1)) + Int64(1) AS t2
physical_plan
ProjectionExec: expr=[t1_id@0 as t1_id, SUM(t2.t2_int * Float64(1)) + Int64(1)@1 as t2_sum]
--CoalesceBatchesExec: target_batch_size=8192
-----HashJoinExec: mode=Partitioned, join_type=Left, on=[(Column { name: "t1_id", index: 0 }, Column { name: "t2_id", index: 1 })]
+----HashJoinExec: mode=Partitioned, join_type=Left, on=[(t1_id@0, t2_id@1)]
------CoalesceBatchesExec: target_batch_size=8192
---------RepartitionExec: partitioning=Hash([Column { name: "t1_id", index: 0 }], 4), input_partitions=4
+--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4
----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0]
------ProjectionExec: expr=[SUM(t2.t2_int * Float64(1))@1 + 1 as SUM(t2.t2_int * Float64(1)) + Int64(1), t2_id@0 as t2_id]
--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int * Float64(1))]
----------CoalesceBatchesExec: target_batch_size=8192
-------------RepartitionExec: partitioning=Hash([Column { name: "t2_id", index: 0 }], 4), input_partitions=4
+------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4
--------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int * Float64(1))]
----------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0]
@@ -246,16 +246,16 @@ Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int) AS t2_sum
physical_plan
ProjectionExec: expr=[t1_id@0 as t1_id, SUM(t2.t2_int)@1 as t2_sum]
--CoalesceBatchesExec: target_batch_size=8192
-----HashJoinExec: mode=Partitioned, join_type=Left, on=[(Column { name: "t1_id", index: 0 }, Column { name: "t2_id", index: 1 })]
+----HashJoinExec: mode=Partitioned, join_type=Left, on=[(t1_id@0, t2_id@1)]
------CoalesceBatchesExec: target_batch_size=8192
---------RepartitionExec: partitioning=Hash([Column { name: "t1_id", index: 0 }], 4), input_partitions=4
+--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4
----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0]
------CoalesceBatchesExec: target_batch_size=8192
---------RepartitionExec: partitioning=Hash([Column { name: "t2_id", index: 1 }], 4), input_partitions=4
+--------RepartitionExec: partitioning=Hash([t2_id@1], 4), input_partitions=4
----------ProjectionExec: expr=[SUM(t2.t2_int)@2 as SUM(t2.t2_int), t2_id@0 as t2_id]
------------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id, Utf8("a")@1 as Utf8("a")], aggr=[SUM(t2.t2_int)]
--------------CoalesceBatchesExec: target_batch_size=8192
-----------------RepartitionExec: partitioning=Hash([Column { name: "t2_id", index: 0 }, Column { name: "Utf8(\"a\")", index: 1 }], 4), input_partitions=4
+----------------RepartitionExec: partitioning=Hash([t2_id@0, Utf8("a")@1], 4), input_partitions=4
------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id, a as Utf8("a")], aggr=[SUM(t2.t2_int)]
--------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0]
@@ -283,16 +283,16 @@ Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int) AS t2_sum
physical_plan
ProjectionExec: expr=[t1_id@0 as t1_id, SUM(t2.t2_int)@1 as t2_sum]
--CoalesceBatchesExec: target_batch_size=8192
-----HashJoinExec: mode=Partitioned, join_type=Left, on=[(Column { name: "t1_id", index: 0 }, Column { name: "t2_id", index: 1 })]
+----HashJoinExec: mode=Partitioned, join_type=Left, on=[(t1_id@0, t2_id@1)]
------CoalesceBatchesExec: target_batch_size=8192
---------RepartitionExec: partitioning=Hash([Column { name: "t1_id", index: 0 }], 4), input_partitions=4
+--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4
----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0]
------ProjectionExec: expr=[SUM(t2.t2_int)@1 as SUM(t2.t2_int), t2_id@0 as t2_id]
--------CoalesceBatchesExec: target_batch_size=8192
----------FilterExec: SUM(t2.t2_int)@1 < 3
------------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)]
--------------CoalesceBatchesExec: target_batch_size=8192
-----------------RepartitionExec: partitioning=Hash([Column { name: "t2_id", index: 0 }], 4), input_partitions=4
+----------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4
------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)]
--------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0]
diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q1.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q1.slt.part
index 8a0e5fd322..776c13e928 100644
--- a/datafusion/core/tests/sqllogictests/test_files/tpch/q1.slt.part
+++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q1.slt.part
@@ -52,7 +52,7 @@ SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS
----ProjectionExec: expr=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus, SUM(lineitem.l_quantity)@2 as sum_qty, SUM(lineitem.l_extendedprice)@3 as sum_base_price, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@4 as sum_disc_price, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax)@5 as sum_charge, AVG(lineitem.l_quantity)@6 as avg_qty, AVG(lineitem.l_extendedprice)@7 as avg_price, AVG(lineitem.l_discount)@8 as avg_d [...]
------AggregateExec: mode=FinalPartitioned, gby=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus], aggr=[SUM(lineitem.l_quantity), SUM(lineitem.l_extendedprice), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), AVG(lineitem.l_quantity), AVG(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(UInt8(1))]
--------CoalesceBatchesExec: target_batch_size=8192
-----------RepartitionExec: partitioning=Hash([Column { name: "l_returnflag", index: 0 }, Column { name: "l_linestatus", index: 1 }], 4), input_partitions=4
+----------RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 4), input_partitions=4
------------AggregateExec: mode=Partial, gby=[l_returnflag@5 as l_returnflag, l_linestatus@6 as l_linestatus], aggr=[SUM(lineitem.l_quantity), SUM(lineitem.l_extendedprice), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), AVG(lineitem.l_quantity), AVG(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(UInt8(1))]
--------------ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)Decimal128(Some(1),20,0) - lineitem.l_discountlineitem.l_discountDecimal128(Some(1),20,0)lineitem.l_extendedprice, l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus]
----------------CoalesceBatchesExec: target_batch_size=8192
diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q10.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q10.slt.part
index 82220d7c93..d2e06d5ff6 100644
--- a/datafusion/core/tests/sqllogictests/test_files/tpch/q10.slt.part
+++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q10.slt.part
@@ -76,41 +76,41 @@ GlobalLimitExec: skip=0, fetch=10
------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@7 as revenue, c_acctbal@2 as c_acctbal, n_name@4 as n_name, c_address@5 as c_address, c_phone@3 as c_phone, c_comment@6 as c_comment]
--------AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@2 as c_acctbal, c_phone@3 as c_phone, n_name@4 as n_name, c_address@5 as c_address, c_comment@6 as c_comment], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]
----------CoalesceBatchesExec: target_batch_size=8192
-------------RepartitionExec: partitioning=Hash([Column { name: "c_custkey", index: 0 }, Column { name: "c_name", index: 1 }, Column { name: "c_acctbal", index: 2 }, Column { name: "c_phone", index: 3 }, Column { name: "n_name", index: 4 }, Column { name: "c_address", index: 5 }, Column { name: "c_comment", index: 6 }], 4), input_partitions=4
+------------RepartitionExec: partitioning=Hash([c_custkey@0, c_name@1, c_acctbal@2, c_phone@3, n_name@4, c_address@5, c_comment@6], 4), input_partitions=4
--------------AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@4 as c_acctbal, c_phone@3 as c_phone, n_name@8 as n_name, c_address@2 as c_address, c_comment@5 as c_comment], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]
----------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_address@2 as c_address, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, l_extendedprice@7 as l_extendedprice, l_discount@8 as l_discount, n_name@10 as n_name]
------------------CoalesceBatchesExec: target_batch_size=8192
---------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_nationkey", index: 3 }, Column { name: "n_nationkey", index: 0 })]
+--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)]
----------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------RepartitionExec: partitioning=Hash([Column { name: "c_nationkey", index: 3 }], 4), input_partitions=4
+------------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4
--------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_address@2 as c_address, c_nationkey@3 as c_nationkey, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, l_extendedprice@9 as l_extendedprice, l_discount@10 as l_discount]
----------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_orderkey", index: 7 }, Column { name: "l_orderkey", index: 0 })]
+------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@7, l_orderkey@0)]
--------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------RepartitionExec: partitioning=Hash([Column { name: "o_orderkey", index: 7 }], 4), input_partitions=4
+----------------------------------RepartitionExec: partitioning=Hash([o_orderkey@7], 4), input_partitions=4
------------------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_address@2 as c_address, c_nationkey@3 as c_nationkey, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, o_orderkey@7 as o_orderkey]
--------------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 1 })]
+----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)]
------------------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4), input_partitions=4
+--------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4
----------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment], has_header=false
------------------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4), input_partitions=4
+--------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4
----------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey]
------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
--------------------------------------------------FilterExec: o_orderdate@2 >= 8674 AND o_orderdate@2 < 8766
----------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false
--------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4), input_partitions=4
+----------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4
------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount]
--------------------------------------CoalesceBatchesExec: target_batch_size=8192
----------------------------------------FilterExec: l_returnflag@3 = R
------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
--------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], has_header=false
----------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=4
+------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4
--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false
diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q11.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q11.slt.part
index b3a462baaa..af29708c67 100644
--- a/datafusion/core/tests/sqllogictests/test_files/tpch/q11.slt.part
+++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q11.slt.part
@@ -78,29 +78,29 @@ GlobalLimitExec: skip=0, fetch=10
--SortPreservingMergeExec: [value@1 DESC]
----SortExec: fetch=10, expr=[value@1 DESC]
------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@1 as value]
---------NestedLoopJoinExec: join_type=Inner, filter=BinaryExpr { left: CastExpr { expr: Column { name: "SUM(partsupp.ps_supplycost * partsupp.ps_availqty)", index: 0 }, cast_type: Decimal128(38, 15), cast_options: CastOptions { safe: false, format_options: FormatOptions { safe: true, null: "", date_format: None, datetime_format: None, timestamp_format: None, timestamp_tz_format: None, time_format: None } } }, op: Gt, right: Column { name: "SUM(partsupp.ps_supplycost * partsupp.ps_availqt [...]
+--------NestedLoopJoinExec: join_type=Inner, filter=CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Decimal128(38, 15)) > SUM(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)@1
----------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)]
------------CoalesceBatchesExec: target_batch_size=8192
---------------RepartitionExec: partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4), input_partitions=4
+--------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4
----------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)]
------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost]
--------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 3 }, Column { name: "n_nationkey", index: 0 })]
+----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)]
------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------RepartitionExec: partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 4), input_partitions=4
+--------------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4
----------------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_availqty@2 as ps_availqty, ps_supplycost@3 as ps_supplycost, s_nationkey@5 as s_nationkey]
------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "ps_suppkey", index: 1 }, Column { name: "s_suppkey", index: 0 })]
+--------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)]
----------------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------------RepartitionExec: partitioning=Hash([Column { name: "ps_suppkey", index: 1 }], 4), input_partitions=4
+------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4
--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl]]}, projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost], has_header=false
----------------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------------RepartitionExec: partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4), input_partitions=4
+------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4
--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false
------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=4
+--------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4
----------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey]
------------------------------CoalesceBatchesExec: target_batch_size=8192
--------------------------------FilterExec: n_name@1 = GERMANY
@@ -112,22 +112,22 @@ GlobalLimitExec: skip=0, fetch=10
----------------AggregateExec: mode=Partial, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)]
------------------ProjectionExec: expr=[ps_availqty@0 as ps_availqty, ps_supplycost@1 as ps_supplycost]
--------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 2 }, Column { name: "n_nationkey", index: 0 })]
+----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)]
------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------RepartitionExec: partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 4), input_partitions=4
+--------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4
----------------------------ProjectionExec: expr=[ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost, s_nationkey@4 as s_nationkey]
------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "ps_suppkey", index: 0 }, Column { name: "s_suppkey", index: 0 })]
+--------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)]
----------------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------------RepartitionExec: partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 4), input_partitions=4
+------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4
--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl]]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], has_header=false
----------------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------------RepartitionExec: partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4), input_partitions=4
+------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4
--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false
------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=4
+--------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4
----------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey]
------------------------------CoalesceBatchesExec: target_batch_size=8192
--------------------------------FilterExec: n_name@1 = GERMANY
diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q12.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q12.slt.part
index fdada35952..dda4ff6fd7 100644
--- a/datafusion/core/tests/sqllogictests/test_files/tpch/q12.slt.part
+++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q12.slt.part
@@ -64,20 +64,20 @@ SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST]
----ProjectionExec: expr=[l_shipmode@0 as l_shipmode, SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@1 as high_line_count, SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@2 as low_line_count]
------AggregateExec: mode=FinalPartitioned, gby=[l_shipmode@0 as l_shipmode], aggr=[SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)]
--------CoalesceBatchesExec: target_batch_size=8192
-----------RepartitionExec: partitioning=Hash([Column { name: "l_shipmode", index: 0 }], 4), input_partitions=4
+----------RepartitionExec: partitioning=Hash([l_shipmode@0], 4), input_partitions=4
------------AggregateExec: mode=Partial, gby=[l_shipmode@0 as l_shipmode], aggr=[SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)]
--------------ProjectionExec: expr=[l_shipmode@1 as l_shipmode, o_orderpriority@3 as o_orderpriority]
----------------CoalesceBatchesExec: target_batch_size=8192
-------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderkey", index: 0 })]
+------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)]
--------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4), input_partitions=4
+----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4
------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_shipmode@4 as l_shipmode]
--------------------------CoalesceBatchesExec: target_batch_size=8192
----------------------------FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_commitdate@2 < l_receiptdate@3 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 8766 AND l_receiptdate@3 < 9131
------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], has_header=false
--------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------RepartitionExec: partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4), input_partitions=4
+----------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4
------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl]]}, projection=[o_orderkey, o_orderpriority], has_header=false
diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q13.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q13.slt.part
index 64848f41fc..7e5be14271 100644
--- a/datafusion/core/tests/sqllogictests/test_files/tpch/q13.slt.part
+++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q13.slt.part
@@ -61,20 +61,20 @@ GlobalLimitExec: skip=0, fetch=10
------ProjectionExec: expr=[c_count@0 as c_count, COUNT(UInt8(1))@1 as custdist]
--------AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[COUNT(UInt8(1))]
----------CoalesceBatchesExec: target_batch_size=8192
-------------RepartitionExec: partitioning=Hash([Column { name: "c_count", index: 0 }], 4), input_partitions=4
+------------RepartitionExec: partitioning=Hash([c_count@0], 4), input_partitions=4
--------------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[COUNT(UInt8(1))]
----------------ProjectionExec: expr=[COUNT(orders.o_orderkey)@0 as c_count]
------------------ProjectionExec: expr=[COUNT(orders.o_orderkey)@1 as COUNT(orders.o_orderkey)]
--------------------AggregateExec: mode=Single, gby=[c_custkey@0 as c_custkey], aggr=[COUNT(orders.o_orderkey)]
----------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, o_orderkey@1 as o_orderkey]
------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 1 })]
+--------------------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)]
----------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------RepartitionExec: partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4), input_partitions=4
+------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4
--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], has_header=false
----------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------RepartitionExec: partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4), input_partitions=4
+------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4
--------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey]
----------------------------------CoalesceBatchesExec: target_batch_size=8192
------------------------------------FilterExec: o_comment@2 NOT LIKE %special%requests%
diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q14.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q14.slt.part
index 08f4c4eb43..bff3ccf4fa 100644
--- a/datafusion/core/tests/sqllogictests/test_files/tpch/q14.slt.part
+++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q14.slt.part
@@ -47,16 +47,16 @@ ProjectionExec: expr=[100 * CAST(SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%")
------AggregateExec: mode=Partial, gby=[], aggr=[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]
--------ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)Decimal128(Some(1),20,0) - lineitem.l_discountlineitem.l_discountDecimal128(Some(1),20,0)lineitem.l_extendedprice, p_type@4 as p_type]
----------CoalesceBatchesExec: target_batch_size=8192
-------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_partkey", index: 0 }, Column { name: "p_partkey", index: 0 })]
+------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)]
--------------CoalesceBatchesExec: target_batch_size=8192
-----------------RepartitionExec: partitioning=Hash([Column { name: "l_partkey", index: 0 }], 4), input_partitions=4
+----------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4
------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount]
--------------------CoalesceBatchesExec: target_batch_size=8192
----------------------FilterExec: l_shipdate@3 >= 9374 AND l_shipdate@3 < 9404
------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], has_header=false
--------------CoalesceBatchesExec: target_batch_size=8192
-----------------RepartitionExec: partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4), input_partitions=4
+----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4
------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false
diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q15.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q15.slt.part
index 1e76e63cd3..700ecad7c6 100644
--- a/datafusion/core/tests/sqllogictests/test_files/tpch/q15.slt.part
+++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q15.slt.part
@@ -76,20 +76,20 @@ SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST]
--SortExec: expr=[s_suppkey@0 ASC NULLS LAST]
----ProjectionExec: expr=[s_suppkey@0 as s_suppkey, s_name@1 as s_name, s_address@2 as s_address, s_phone@3 as s_phone, total_revenue@4 as total_revenue]
------CoalesceBatchesExec: target_batch_size=8192
---------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "total_revenue", index: 4 }, Column { name: "MAX(total_revenue)", index: 0 })]
+--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(total_revenue@4, MAX(total_revenue)@0)]
----------CoalesceBatchesExec: target_batch_size=8192
-------------RepartitionExec: partitioning=Hash([Column { name: "total_revenue", index: 4 }], 4), input_partitions=4
+------------RepartitionExec: partitioning=Hash([total_revenue@4], 4), input_partitions=4
--------------ProjectionExec: expr=[s_suppkey@0 as s_suppkey, s_name@1 as s_name, s_address@2 as s_address, s_phone@3 as s_phone, total_revenue@5 as total_revenue]
----------------CoalesceBatchesExec: target_batch_size=8192
-------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_suppkey", index: 0 }, Column { name: "supplier_no", index: 0 })]
+------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, supplier_no@0)]
--------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------RepartitionExec: partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4), input_partitions=4
+----------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4
------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_phone], has_header=false
--------------------ProjectionExec: expr=[l_suppkey@0 as supplier_no, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue]
----------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]
------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------RepartitionExec: partitioning=Hash([Column { name: "l_suppkey", index: 0 }], 4), input_partitions=4
+--------------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4
----------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]
------------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount]
--------------------------------CoalesceBatchesExec: target_batch_size=8192
@@ -97,7 +97,7 @@ SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST]
------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
--------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false
----------CoalesceBatchesExec: target_batch_size=8192
-------------RepartitionExec: partitioning=Hash([Column { name: "MAX(total_revenue)", index: 0 }], 4), input_partitions=1
+------------RepartitionExec: partitioning=Hash([MAX(total_revenue)@0], 4), input_partitions=1
--------------AggregateExec: mode=Final, gby=[], aggr=[MAX(total_revenue)]
----------------CoalescePartitionsExec
------------------AggregateExec: mode=Partial, gby=[], aggr=[MAX(total_revenue)]
@@ -105,7 +105,7 @@ SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST]
----------------------ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]
------------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]
--------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------RepartitionExec: partitioning=Hash([Column { name: "l_suppkey", index: 0 }], 4), input_partitions=4
+----------------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4
------------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]
--------------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount]
----------------------------------CoalesceBatchesExec: target_batch_size=8192
diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q16.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q16.slt.part
index 4f4316b084..677db0329c 100644
--- a/datafusion/core/tests/sqllogictests/test_files/tpch/q16.slt.part
+++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q16.slt.part
@@ -72,31 +72,31 @@ GlobalLimitExec: skip=0, fetch=10
------ProjectionExec: expr=[group_alias_0@0 as part.p_brand, group_alias_1@1 as part.p_type, group_alias_2@2 as part.p_size, COUNT(alias1)@3 as supplier_cnt]
--------AggregateExec: mode=FinalPartitioned, gby=[group_alias_0@0 as group_alias_0, group_alias_1@1 as group_alias_1, group_alias_2@2 as group_alias_2], aggr=[COUNT(alias1)]
----------CoalesceBatchesExec: target_batch_size=8192
-------------RepartitionExec: partitioning=Hash([Column { name: "group_alias_0", index: 0 }, Column { name: "group_alias_1", index: 1 }, Column { name: "group_alias_2", index: 2 }], 4), input_partitions=4
+------------RepartitionExec: partitioning=Hash([group_alias_0@0, group_alias_1@1, group_alias_2@2], 4), input_partitions=4
--------------AggregateExec: mode=Partial, gby=[group_alias_0@0 as group_alias_0, group_alias_1@1 as group_alias_1, group_alias_2@2 as group_alias_2], aggr=[COUNT(alias1)]
----------------AggregateExec: mode=FinalPartitioned, gby=[group_alias_0@0 as group_alias_0, group_alias_1@1 as group_alias_1, group_alias_2@2 as group_alias_2, alias1@3 as alias1], aggr=[]
------------------CoalesceBatchesExec: target_batch_size=8192
---------------------RepartitionExec: partitioning=Hash([Column { name: "group_alias_0", index: 0 }, Column { name: "group_alias_1", index: 1 }, Column { name: "group_alias_2", index: 2 }, Column { name: "alias1", index: 3 }], 4), input_partitions=4
+--------------------RepartitionExec: partitioning=Hash([group_alias_0@0, group_alias_1@1, group_alias_2@2, alias1@3], 4), input_partitions=4
----------------------AggregateExec: mode=Partial, gby=[p_brand@1 as group_alias_0, p_type@2 as group_alias_1, p_size@3 as group_alias_2, ps_suppkey@0 as alias1], aggr=[]
------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(Column { name: "ps_suppkey", index: 0 }, Column { name: "s_suppkey", index: 0 })]
+--------------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(ps_suppkey@0, s_suppkey@0)]
----------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------RepartitionExec: partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 4), input_partitions=4
+------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4
--------------------------------ProjectionExec: expr=[ps_suppkey@1 as ps_suppkey, p_brand@3 as p_brand, p_type@4 as p_type, p_size@5 as p_size]
----------------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "ps_partkey", index: 0 }, Column { name: "p_partkey", index: 0 })]
+------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, p_partkey@0)]
--------------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------------RepartitionExec: partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4), input_partitions=4
+----------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4
------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
--------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl]]}, projection=[ps_partkey, ps_suppkey], has_header=false
--------------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------------RepartitionExec: partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4), input_partitions=4
+----------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4
------------------------------------------CoalesceBatchesExec: target_batch_size=8192
--------------------------------------------FilterExec: p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(49) }, Literal { value: Int32(14) }, Literal { value: Int32(23) }, Literal { value: Int32(45) }, Literal { value: Int32(19) }, Literal { value: Int32(3) }, Literal { value: Int32(36) }, Literal { value: Int32(9) }])
----------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_type, p_size], has_header=false
----------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------RepartitionExec: partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4), input_partitions=4
+------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4
--------------------------------ProjectionExec: expr=[s_suppkey@0 as s_suppkey]
----------------------------------CoalesceBatchesExec: target_batch_size=8192
------------------------------------FilterExec: s_comment@1 LIKE %Customer%Complaints%
diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q17.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q17.slt.part
index 364369016b..ca90c78280 100644
--- a/datafusion/core/tests/sqllogictests/test_files/tpch/q17.slt.part
+++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q17.slt.part
@@ -57,18 +57,18 @@ ProjectionExec: expr=[CAST(SUM(lineitem.l_extendedprice)@0 AS Float64) / 7 as av
------AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice)]
--------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice]
----------CoalesceBatchesExec: target_batch_size=8192
-------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "p_partkey", index: 2 }, Column { name: "l_partkey", index: 1 })], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * AVG(lineitem.l_quantity)@1
+------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * AVG(lineitem.l_quantity)@1
--------------CoalesceBatchesExec: target_batch_size=8192
-----------------RepartitionExec: partitioning=Hash([Column { name: "p_partkey", index: 2 }], 4), input_partitions=4
+----------------RepartitionExec: partitioning=Hash([p_partkey@2], 4), input_partitions=4
------------------ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, p_partkey@3 as p_partkey]
--------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_partkey", index: 0 }, Column { name: "p_partkey", index: 0 })]
+----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)]
------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------RepartitionExec: partitioning=Hash([Column { name: "l_partkey", index: 0 }], 4), input_partitions=4
+--------------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4
----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_partkey, l_quantity, l_extendedprice], has_header=false
------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------RepartitionExec: partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4), input_partitions=4
+--------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4
----------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey]
------------------------------CoalesceBatchesExec: target_batch_size=8192
--------------------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX
@@ -77,7 +77,7 @@ ProjectionExec: expr=[CAST(SUM(lineitem.l_extendedprice)@0 AS Float64) / 7 as av
--------------ProjectionExec: expr=[CAST(0.2 * CAST(AVG(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * AVG(lineitem.l_quantity), l_partkey@0 as l_partkey]
----------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)]
------------------CoalesceBatchesExec: target_batch_size=8192
---------------------RepartitionExec: partitioning=Hash([Column { name: "l_partkey", index: 0 }], 4), input_partitions=4
+--------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4
----------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)]
------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_partkey, l_quantity], has_header=false
diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q18.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q18.slt.part
index 01d683ffe4..3570b7bd89 100644
--- a/datafusion/core/tests/sqllogictests/test_files/tpch/q18.slt.part
+++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q18.slt.part
@@ -71,28 +71,28 @@ SortPreservingMergeExec: [o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST]
--SortExec: expr=[o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST]
----AggregateExec: mode=FinalPartitioned, gby=[c_name@0 as c_name, c_custkey@1 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@3 as o_orderdate, o_totalprice@4 as o_totalprice], aggr=[SUM(lineitem.l_quantity)]
------CoalesceBatchesExec: target_batch_size=8192
---------RepartitionExec: partitioning=Hash([Column { name: "c_name", index: 0 }, Column { name: "c_custkey", index: 1 }, Column { name: "o_orderkey", index: 2 }, Column { name: "o_orderdate", index: 3 }, Column { name: "o_totalprice", index: 4 }], 4), input_partitions=4
+--------RepartitionExec: partitioning=Hash([c_name@0, c_custkey@1, o_orderkey@2, o_orderdate@3, o_totalprice@4], 4), input_partitions=4
----------AggregateExec: mode=Partial, gby=[c_name@1 as c_name, c_custkey@0 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@4 as o_orderdate, o_totalprice@3 as o_totalprice], aggr=[SUM(lineitem.l_quantity)]
------------CoalesceBatchesExec: target_batch_size=8192
---------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: "o_orderkey", index: 2 }, Column { name: "l_orderkey", index: 0 })]
+--------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@2, l_orderkey@0)]
----------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, o_orderkey@2 as o_orderkey, o_totalprice@3 as o_totalprice, o_orderdate@4 as o_orderdate, l_quantity@6 as l_quantity]
------------------CoalesceBatchesExec: target_batch_size=8192
---------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_orderkey", index: 2 }, Column { name: "l_orderkey", index: 0 })]
+--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@2, l_orderkey@0)]
----------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------RepartitionExec: partitioning=Hash([Column { name: "o_orderkey", index: 2 }], 4), input_partitions=4
+------------------------RepartitionExec: partitioning=Hash([o_orderkey@2], 4), input_partitions=4
--------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, o_orderkey@2 as o_orderkey, o_totalprice@4 as o_totalprice, o_orderdate@5 as o_orderdate]
----------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 1 })]
+------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)]
--------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------RepartitionExec: partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4), input_partitions=4
+----------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4
------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
--------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name], has_header=false
--------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------RepartitionExec: partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4), input_partitions=4
+----------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4
------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
--------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl]]}, projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate], has_header=false
----------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4), input_partitions=4
+------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4
--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_orderkey, l_quantity], has_header=false
----------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey]
@@ -100,7 +100,7 @@ SortPreservingMergeExec: [o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST]
--------------------FilterExec: SUM(lineitem.l_quantity)@1 > Some(30000),25,2
----------------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[SUM(lineitem.l_quantity)]
------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4), input_partitions=4
+--------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4
----------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[SUM(lineitem.l_quantity)]
------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_orderkey, l_quantity], has_header=false
diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q19.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q19.slt.part
index 1a91fed124..0906aa4aad 100644
--- a/datafusion/core/tests/sqllogictests/test_files/tpch/q19.slt.part
+++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q19.slt.part
@@ -70,16 +70,16 @@ ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_disco
------AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]
--------ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount]
----------CoalesceBatchesExec: target_batch_size=8192
-------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_partkey", index: 0 }, Column { name: "p_partkey", index: 0 })], filter=p_brand@1 = Brand#12 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND Use p_container@3 IN (S [...]
+------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { va [...]
--------------CoalesceBatchesExec: target_batch_size=8192
-----------------RepartitionExec: partitioning=Hash([Column { name: "l_partkey", index: 0 }], 4), input_partitions=4
+----------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4
------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount]
--------------------CoalesceBatchesExec: target_batch_size=8192
----------------------FilterExec: (l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2) AND (l_shipmode@5 = AIR OR l_shipmode@5 = AIR REG) AND l_shipinstruct@4 = DELIVER IN PERSON
------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], has_header=false
--------------CoalesceBatchesExec: target_batch_size=8192
-----------------RepartitionExec: partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4), input_partitions=4
+----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4
------------------CoalesceBatchesExec: target_batch_size=8192
--------------------FilterExec: (p_brand@1 = Brand#12 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { value: Utf8("MED BOX") }, Literal { value: Utf8("MED PKG") }, Literal { value: Utf8("MED PACK") }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#3 [...]
----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q2.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q2.slt.part
index ab9faeeb76..4ad1ed7293 100644
--- a/datafusion/core/tests/sqllogictests/test_files/tpch/q2.slt.part
+++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q2.slt.part
@@ -105,87 +105,87 @@ GlobalLimitExec: skip=0, fetch=10
----SortExec: fetch=10, expr=[s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST]
------ProjectionExec: expr=[s_acctbal@5 as s_acctbal, s_name@2 as s_name, n_name@8 as n_name, p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_address@3 as s_address, s_phone@4 as s_phone, s_comment@6 as s_comment]
--------CoalesceBatchesExec: target_batch_size=8192
-----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "p_partkey", index: 0 }, Column { name: "ps_partkey", index: 1 }), (Column { name: "ps_supplycost", index: 7 }, Column { name: "MIN(partsupp.ps_supplycost)", index: 0 })]
+----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@1), (ps_supplycost@7, MIN(partsupp.ps_supplycost)@0)]
------------CoalesceBatchesExec: target_batch_size=8192
---------------RepartitionExec: partitioning=Hash([Column { name: "p_partkey", index: 0 }, Column { name: "ps_supplycost", index: 7 }], 4), input_partitions=4
+--------------RepartitionExec: partitioning=Hash([p_partkey@0, ps_supplycost@7], 4), input_partitions=4
----------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_name@2 as s_name, s_address@3 as s_address, s_phone@4 as s_phone, s_acctbal@5 as s_acctbal, s_comment@6 as s_comment, ps_supplycost@7 as ps_supplycost, n_name@8 as n_name]
------------------CoalesceBatchesExec: target_batch_size=8192
---------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "n_regionkey", index: 9 }, Column { name: "r_regionkey", index: 0 })]
+--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@9, r_regionkey@0)]
----------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------RepartitionExec: partitioning=Hash([Column { name: "n_regionkey", index: 9 }], 4), input_partitions=4
+------------------------RepartitionExec: partitioning=Hash([n_regionkey@9], 4), input_partitions=4
--------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_name@2 as s_name, s_address@3 as s_address, s_phone@5 as s_phone, s_acctbal@6 as s_acctbal, s_comment@7 as s_comment, ps_supplycost@8 as ps_supplycost, n_name@10 as n_name, n_regionkey@11 as n_regionkey]
----------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 4 }, Column { name: "n_nationkey", index: 0 })]
+------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@4, n_nationkey@0)]
--------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------RepartitionExec: partitioning=Hash([Column { name: "s_nationkey", index: 4 }], 4), input_partitions=4
+----------------------------------RepartitionExec: partitioning=Hash([s_nationkey@4], 4), input_partitions=4
------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_name@5 as s_name, s_address@6 as s_address, s_nationkey@7 as s_nationkey, s_phone@8 as s_phone, s_acctbal@9 as s_acctbal, s_comment@10 as s_comment, ps_supplycost@3 as ps_supplycost]
--------------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "ps_suppkey", index: 2 }, Column { name: "s_suppkey", index: 0 })]
+----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@2, s_suppkey@0)]
------------------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "ps_suppkey", index: 2 }], 4), input_partitions=4
+--------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@2], 4), input_partitions=4
----------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, ps_suppkey@3 as ps_suppkey, ps_supplycost@4 as ps_supplycost]
------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "p_partkey", index: 0 }, Column { name: "ps_partkey", index: 0 })]
+--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)]
----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4), input_partitions=4
+------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4
--------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr]
----------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
------------------------------------------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS
--------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
----------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_mfgr, p_type, p_size], has_header=false
----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4), input_partitions=4
+------------------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4
--------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
----------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false
------------------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4), input_partitions=4
+--------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4
----------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], has_header=false
--------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=4
+----------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4
------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
--------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], has_header=false
----------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------RepartitionExec: partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 4), input_partitions=4
+------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4
--------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey]
----------------------------CoalesceBatchesExec: target_batch_size=8192
------------------------------FilterExec: r_name@1 = EUROPE
--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false
------------CoalesceBatchesExec: target_batch_size=8192
---------------RepartitionExec: partitioning=Hash([Column { name: "ps_partkey", index: 1 }, Column { name: "MIN(partsupp.ps_supplycost)", index: 0 }], 4), input_partitions=4
+--------------RepartitionExec: partitioning=Hash([ps_partkey@1, MIN(partsupp.ps_supplycost)@0], 4), input_partitions=4
----------------ProjectionExec: expr=[MIN(partsupp.ps_supplycost)@1 as MIN(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey]
------------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)]
--------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------RepartitionExec: partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4), input_partitions=4
+----------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4
------------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)]
--------------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_supplycost@1 as ps_supplycost]
----------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "n_regionkey", index: 2 }, Column { name: "r_regionkey", index: 0 })]
+------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)]
--------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------RepartitionExec: partitioning=Hash([Column { name: "n_regionkey", index: 2 }], 4), input_partitions=4
+----------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4
------------------------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_supplycost@1 as ps_supplycost, n_regionkey@4 as n_regionkey]
--------------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 2 }, Column { name: "n_nationkey", index: 0 })]
+----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)]
------------------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 4), input_partitions=4
+--------------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4
----------------------------------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_supplycost@2 as ps_supplycost, s_nationkey@4 as s_nationkey]
------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "ps_suppkey", index: 1 }, Column { name: "s_suppkey", index: 0 })]
+--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)]
----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "ps_suppkey", index: 1 }], 4), input_partitions=4
+------------------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4
--------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
----------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false
----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4), input_partitions=4
+------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4
--------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
----------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false
------------------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=4
+--------------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4
----------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false
--------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------RepartitionExec: partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 4), input_partitions=4
+----------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4
------------------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey]
--------------------------------------CoalesceBatchesExec: target_batch_size=8192
----------------------------------------FilterExec: r_name@1 = EUROPE
diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q20.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q20.slt.part
index ef0ec5dfb3..ec9bb4ee41 100644
--- a/datafusion/core/tests/sqllogictests/test_files/tpch/q20.slt.part
+++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q20.slt.part
@@ -85,38 +85,38 @@ SortPreservingMergeExec: [s_name@0 ASC NULLS LAST]
--SortExec: expr=[s_name@0 ASC NULLS LAST]
----ProjectionExec: expr=[s_name@1 as s_name, s_address@2 as s_address]
------CoalesceBatchesExec: target_batch_size=8192
---------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: "s_suppkey", index: 0 }, Column { name: "ps_suppkey", index: 0 })]
+--------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(s_suppkey@0, ps_suppkey@0)]
----------CoalesceBatchesExec: target_batch_size=8192
-------------RepartitionExec: partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4), input_partitions=4
+------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4
--------------ProjectionExec: expr=[s_suppkey@0 as s_suppkey, s_name@1 as s_name, s_address@2 as s_address]
----------------CoalesceBatchesExec: target_batch_size=8192
-------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 3 }, Column { name: "n_nationkey", index: 0 })]
+------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)]
--------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------RepartitionExec: partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 4), input_partitions=4
+----------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4
------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey], has_header=false
--------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=4
+----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4
------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey]
--------------------------CoalesceBatchesExec: target_batch_size=8192
----------------------------FilterExec: n_name@1 = CANADA
------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false
----------CoalesceBatchesExec: target_batch_size=8192
-------------RepartitionExec: partitioning=Hash([Column { name: "ps_suppkey", index: 0 }], 4), input_partitions=4
+------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4
--------------ProjectionExec: expr=[ps_suppkey@1 as ps_suppkey]
----------------CoalesceBatchesExec: target_batch_size=8192
-------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "ps_partkey", index: 0 }, Column { name: "l_partkey", index: 1 }), (Column { name: "ps_suppkey", index: 1 }, Column { name: "l_suppkey", index: 2 })], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * SUM(lineitem.l_quantity)@1
+------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * SUM(lineitem.l_quantity)@1
--------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------RepartitionExec: partitioning=Hash([Column { name: "ps_partkey", index: 0 }, Column { name: "ps_suppkey", index: 1 }], 4), input_partitions=4
+----------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4
------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: "ps_partkey", index: 0 }, Column { name: "p_partkey", index: 0 })]
+--------------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)]
----------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------RepartitionExec: partitioning=Hash([Column { name: "ps_partkey", index: 0 }], 4), input_partitions=4
+------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4
--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl]]}, projection=[ps_partkey, ps_suppkey, ps_availqty], has_header=false
----------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------RepartitionExec: partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4), input_partitions=4
+------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4
--------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey]
----------------------------------CoalesceBatchesExec: target_batch_size=8192
------------------------------------FilterExec: p_name@1 LIKE forest%
@@ -125,7 +125,7 @@ SortPreservingMergeExec: [s_name@0 ASC NULLS LAST]
--------------------ProjectionExec: expr=[0.5 * CAST(SUM(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * SUM(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey]
----------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[SUM(lineitem.l_quantity)]
------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------RepartitionExec: partitioning=Hash([Column { name: "l_partkey", index: 0 }, Column { name: "l_suppkey", index: 1 }], 4), input_partitions=4
+--------------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4
----------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[SUM(lineitem.l_quantity)]
------------------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey, l_quantity@2 as l_quantity]
--------------------------------CoalesceBatchesExec: target_batch_size=8192
diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q21.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q21.slt.part
index 33bff25609..04c45ec5bb 100644
--- a/datafusion/core/tests/sqllogictests/test_files/tpch/q21.slt.part
+++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q21.slt.part
@@ -95,59 +95,59 @@ SortPreservingMergeExec: [numwait@1 DESC,s_name@0 ASC NULLS LAST]
----ProjectionExec: expr=[s_name@0 as s_name, COUNT(UInt8(1))@1 as numwait]
------AggregateExec: mode=FinalPartitioned, gby=[s_name@0 as s_name], aggr=[COUNT(UInt8(1))]
--------CoalesceBatchesExec: target_batch_size=8192
-----------RepartitionExec: partitioning=Hash([Column { name: "s_name", index: 0 }], 4), input_partitions=4
+----------RepartitionExec: partitioning=Hash([s_name@0], 4), input_partitions=4
------------AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[COUNT(UInt8(1))]
--------------ProjectionExec: expr=[s_name@0 as s_name]
----------------CoalesceBatchesExec: target_batch_size=8192
-------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(Column { name: "l_orderkey", index: 1 }, Column { name: "l_orderkey", index: 0 })], filter=l_suppkey@1 != l_suppkey@0
+------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0
--------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: "l_orderkey", index: 1 }, Column { name: "l_orderkey", index: 0 })], filter=l_suppkey@1 != l_suppkey@0
+----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0
------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 1 }], 4), input_partitions=4
+--------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4
----------------------------ProjectionExec: expr=[s_name@0 as s_name, l_orderkey@2 as l_orderkey, l_suppkey@3 as l_suppkey]
------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 1 }, Column { name: "n_nationkey", index: 0 })]
+--------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@1, n_nationkey@0)]
----------------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------------RepartitionExec: partitioning=Hash([Column { name: "s_nationkey", index: 1 }], 4), input_partitions=4
+------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@1], 4), input_partitions=4
--------------------------------------ProjectionExec: expr=[s_name@0 as s_name, s_nationkey@1 as s_nationkey, l_orderkey@2 as l_orderkey, l_suppkey@3 as l_suppkey]
----------------------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_orderkey", index: 2 }, Column { name: "o_orderkey", index: 0 })]
+------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@2, o_orderkey@0)]
--------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 2 }], 4), input_partitions=4
+----------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@2], 4), input_partitions=4
------------------------------------------------ProjectionExec: expr=[s_name@1 as s_name, s_nationkey@2 as s_nationkey, l_orderkey@3 as l_orderkey, l_suppkey@4 as l_suppkey]
--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_suppkey", index: 0 }, Column { name: "l_suppkey", index: 1 })]
+----------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)]
------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4), input_partitions=4
+--------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4
----------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_nationkey], has_header=false
------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 4), input_partitions=4
+--------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4
----------------------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey]
------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
--------------------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2
----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], has_header=false
--------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4), input_partitions=4
+----------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4
------------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey]
--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
----------------------------------------------------FilterExec: o_orderstatus@1 = F
------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
--------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl]]}, projection=[o_orderkey, o_orderstatus], has_header=false
----------------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------------RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=4
+------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4
--------------------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey]
----------------------------------------CoalesceBatchesExec: target_batch_size=8192
------------------------------------------FilterExec: n_name@1 = SAUDI ARABIA
--------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
----------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false
------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4), input_partitions=4
+--------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4
----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_orderkey, l_suppkey], has_header=false
--------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4), input_partitions=4
+----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4
------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey]
--------------------------CoalesceBatchesExec: target_batch_size=8192
----------------------------FilterExec: l_receiptdate@3 > l_commitdate@2
diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q22.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q22.slt.part
index 35024c4b0d..e5cbc02822 100644
--- a/datafusion/core/tests/sqllogictests/test_files/tpch/q22.slt.part
+++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q22.slt.part
@@ -79,21 +79,21 @@ SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST]
----ProjectionExec: expr=[cntrycode@0 as cntrycode, COUNT(UInt8(1))@1 as numcust, SUM(custsale.c_acctbal)@2 as totacctbal]
------AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[COUNT(UInt8(1)), SUM(custsale.c_acctbal)]
--------CoalesceBatchesExec: target_batch_size=8192
-----------RepartitionExec: partitioning=Hash([Column { name: "cntrycode", index: 0 }], 4), input_partitions=4
+----------RepartitionExec: partitioning=Hash([cntrycode@0], 4), input_partitions=4
------------AggregateExec: mode=Partial, gby=[cntrycode@0 as cntrycode], aggr=[COUNT(UInt8(1)), SUM(custsale.c_acctbal)]
--------------ProjectionExec: expr=[substr(c_phone@0, 1, 2) as cntrycode, c_acctbal@1 as c_acctbal]
-----------------NestedLoopJoinExec: join_type=Inner, filter=BinaryExpr { left: CastExpr { expr: Column { name: "c_acctbal", index: 0 }, cast_type: Decimal128(19, 6), cast_options: CastOptions { safe: false, format_options: FormatOptions { safe: true, null: "", date_format: None, datetime_format: None, timestamp_format: None, timestamp_tz_format: None, time_format: None } } }, op: Gt, right: Column { name: "AVG(customer.c_acctbal)", index: 1 } }
+----------------NestedLoopJoinExec: join_type=Inner, filter=CAST(c_acctbal@0 AS Decimal128(19, 6)) > AVG(customer.c_acctbal)@1
------------------ProjectionExec: expr=[c_phone@1 as c_phone, c_acctbal@2 as c_acctbal]
--------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 0 })]
+----------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)]
------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------RepartitionExec: partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4), input_partitions=4
+--------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4
----------------------------CoalesceBatchesExec: target_batch_size=8192
------------------------------FilterExec: Use substr(c_phone@1, 1, 2) IN (SET) ([Literal { value: Utf8("13") }, Literal { value: Utf8("31") }, Literal { value: Utf8("23") }, Literal { value: Utf8("29") }, Literal { value: Utf8("30") }, Literal { value: Utf8("18") }, Literal { value: Utf8("17") }])
--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_phone, c_acctbal], has_header=false
------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------RepartitionExec: partitioning=Hash([Column { name: "o_custkey", index: 0 }], 4), input_partitions=4
+--------------------------RepartitionExec: partitioning=Hash([o_custkey@0], 4), input_partitions=4
----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl]]}, projection=[o_custkey], has_header=false
------------------AggregateExec: mode=Final, gby=[], aggr=[AVG(customer.c_acctbal)]
diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q3.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q3.slt.part
index 381ea531d6..dc3b150877 100644
--- a/datafusion/core/tests/sqllogictests/test_files/tpch/q3.slt.part
+++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q3.slt.part
@@ -65,31 +65,31 @@ GlobalLimitExec: skip=0, fetch=10
------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@3 as revenue, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority]
--------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]
----------CoalesceBatchesExec: target_batch_size=8192
-------------RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderdate", index: 1 }, Column { name: "o_shippriority", index: 2 }], 4), input_partitions=4
+------------RepartitionExec: partitioning=Hash([l_orderkey@0, o_orderdate@1, o_shippriority@2], 4), input_partitions=4
--------------AggregateExec: mode=Partial, gby=[l_orderkey@2 as l_orderkey, o_orderdate@0 as o_orderdate, o_shippriority@1 as o_shippriority], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]
----------------ProjectionExec: expr=[o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority, l_orderkey@3 as l_orderkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount]
------------------CoalesceBatchesExec: target_batch_size=8192
---------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_orderkey", index: 0 }, Column { name: "l_orderkey", index: 0 })]
+--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)]
----------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------RepartitionExec: partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4), input_partitions=4
+------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4
--------------------------ProjectionExec: expr=[o_orderkey@1 as o_orderkey, o_orderdate@3 as o_orderdate, o_shippriority@4 as o_shippriority]
----------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 1 })]
+------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)]
--------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------RepartitionExec: partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4), input_partitions=4
+----------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4
------------------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey]
--------------------------------------CoalesceBatchesExec: target_batch_size=8192
----------------------------------------FilterExec: c_mktsegment@1 = BUILDING
------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
--------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_mktsegment], has_header=false
--------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------RepartitionExec: partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4), input_partitions=4
+----------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4
------------------------------------CoalesceBatchesExec: target_batch_size=8192
--------------------------------------FilterExec: o_orderdate@2 < 9204
----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl]]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], has_header=false
----------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4), input_partitions=4
+------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4
--------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount]
----------------------------CoalesceBatchesExec: target_batch_size=8192
------------------------------FilterExec: l_shipdate@3 > 9204
diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q4.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q4.slt.part
index f3c428ae78..d3e6afa406 100644
--- a/datafusion/core/tests/sqllogictests/test_files/tpch/q4.slt.part
+++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q4.slt.part
@@ -58,20 +58,20 @@ SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST]
----ProjectionExec: expr=[o_orderpriority@0 as o_orderpriority, COUNT(UInt8(1))@1 as order_count]
------AggregateExec: mode=FinalPartitioned, gby=[o_orderpriority@0 as o_orderpriority], aggr=[COUNT(UInt8(1))]
--------CoalesceBatchesExec: target_batch_size=8192
-----------RepartitionExec: partitioning=Hash([Column { name: "o_orderpriority", index: 0 }], 4), input_partitions=4
+----------RepartitionExec: partitioning=Hash([o_orderpriority@0], 4), input_partitions=4
------------AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[COUNT(UInt8(1))]
--------------ProjectionExec: expr=[o_orderpriority@1 as o_orderpriority]
----------------CoalesceBatchesExec: target_batch_size=8192
-------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: "o_orderkey", index: 0 }, Column { name: "l_orderkey", index: 0 })]
+------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)]
--------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------RepartitionExec: partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4), input_partitions=4
+----------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4
------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_orderpriority@2 as o_orderpriority]
--------------------------CoalesceBatchesExec: target_batch_size=8192
----------------------------FilterExec: o_orderdate@1 >= 8582 AND o_orderdate@1 < 8674
------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl]]}, projection=[o_orderkey, o_orderdate, o_orderpriority], has_header=false
--------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4), input_partitions=4
+----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4
------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey]
--------------------------CoalesceBatchesExec: target_batch_size=8192
----------------------------FilterExec: l_commitdate@1 < l_receiptdate@2
diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q5.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q5.slt.part
index feac9d2a04..898f21d3d5 100644
--- a/datafusion/core/tests/sqllogictests/test_files/tpch/q5.slt.part
+++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q5.slt.part
@@ -72,56 +72,56 @@ SortPreservingMergeExec: [revenue@1 DESC]
----ProjectionExec: expr=[n_name@0 as n_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as revenue]
------AggregateExec: mode=FinalPartitioned, gby=[n_name@0 as n_name], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]
--------CoalesceBatchesExec: target_batch_size=8192
-----------RepartitionExec: partitioning=Hash([Column { name: "n_name", index: 0 }], 4), input_partitions=4
+----------RepartitionExec: partitioning=Hash([n_name@0], 4), input_partitions=4
------------AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]
--------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, n_name@2 as n_name]
----------------CoalesceBatchesExec: target_batch_size=8192
-------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "n_regionkey", index: 3 }, Column { name: "r_regionkey", index: 0 })]
+------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)]
--------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------RepartitionExec: partitioning=Hash([Column { name: "n_regionkey", index: 3 }], 4), input_partitions=4
+----------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4
------------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, n_name@4 as n_name, n_regionkey@5 as n_regionkey]
--------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 2 }, Column { name: "n_nationkey", index: 0 })]
+----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)]
------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------RepartitionExec: partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 4), input_partitions=4
+--------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4
----------------------------------ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@5 as s_nationkey]
------------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_suppkey", index: 1 }, Column { name: "s_suppkey", index: 0 }), (Column { name: "c_nationkey", index: 0 }, Column { name: "s_nationkey", index: 1 })]
+--------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0), (c_nationkey@0, s_nationkey@1)]
----------------------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "l_suppkey", index: 1 }, Column { name: "c_nationkey", index: 0 }], 4), input_partitions=4
+------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1, c_nationkey@0], 4), input_partitions=4
--------------------------------------------ProjectionExec: expr=[c_nationkey@0 as c_nationkey, l_suppkey@3 as l_suppkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount]
----------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_orderkey", index: 1 }, Column { name: "l_orderkey", index: 0 })]
+------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@1, l_orderkey@0)]
--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "o_orderkey", index: 1 }], 4), input_partitions=4
+----------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@1], 4), input_partitions=4
------------------------------------------------------ProjectionExec: expr=[c_nationkey@1 as c_nationkey, o_orderkey@2 as o_orderkey]
--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_custkey", index: 0 }, Column { name: "o_custkey", index: 1 })]
+----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)]
------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4), input_partitions=4
+--------------------------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4
----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false
------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "o_custkey", index: 1 }], 4), input_partitions=4
+--------------------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4
----------------------------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey]
------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
--------------------------------------------------------------------FilterExec: o_orderdate@2 >= 8766 AND o_orderdate@2 < 9131
----------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false
--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4), input_partitions=4
+----------------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4
------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
--------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount], has_header=false
----------------------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "s_suppkey", index: 0 }, Column { name: "s_nationkey", index: 1 }], 4), input_partitions=4
+------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=4
--------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
----------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false
------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=4
+--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4
----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], has_header=false
--------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------RepartitionExec: partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 4), input_partitions=4
+----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4
------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey]
--------------------------CoalesceBatchesExec: target_batch_size=8192
----------------------------FilterExec: r_name@1 = ASIA
diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q7.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q7.slt.part
index dd538ebfe8..741fc1864e 100644
--- a/datafusion/core/tests/sqllogictests/test_files/tpch/q7.slt.part
+++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q7.slt.part
@@ -89,57 +89,57 @@ SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS L
----ProjectionExec: expr=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year, SUM(shipping.volume)@3 as revenue]
------AggregateExec: mode=FinalPartitioned, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[SUM(shipping.volume)]
--------CoalesceBatchesExec: target_batch_size=8192
-----------RepartitionExec: partitioning=Hash([Column { name: "supp_nation", index: 0 }, Column { name: "cust_nation", index: 1 }, Column { name: "l_year", index: 2 }], 4), input_partitions=4
+----------RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 4), input_partitions=4
------------AggregateExec: mode=Partial, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[SUM(shipping.volume)]
--------------ProjectionExec: expr=[n_name@4 as supp_nation, n_name@6 as cust_nation, date_part(YEAR, l_shipdate@2) as l_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume]
----------------CoalesceBatchesExec: target_batch_size=8192
-------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_nationkey", index: 3 }, Column { name: "n_nationkey", index: 0 })], filter=n_name@0 = FRANCE AND n_name@1 = GERMANY OR n_name@0 = GERMANY AND n_name@1 = FRANCE
+------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], filter=n_name@0 = FRANCE AND n_name@1 = GERMANY OR n_name@0 = GERMANY AND n_name@1 = FRANCE
--------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------RepartitionExec: partitioning=Hash([Column { name: "c_nationkey", index: 3 }], 4), input_partitions=4
+----------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4
------------------------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_shipdate@3 as l_shipdate, c_nationkey@4 as c_nationkey, n_name@6 as n_name]
--------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 0 }, Column { name: "n_nationkey", index: 0 })]
+----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@0, n_nationkey@0)]
------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------RepartitionExec: partitioning=Hash([Column { name: "s_nationkey", index: 0 }], 4), input_partitions=4
+--------------------------------RepartitionExec: partitioning=Hash([s_nationkey@0], 4), input_partitions=4
----------------------------------ProjectionExec: expr=[s_nationkey@0 as s_nationkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_shipdate@3 as l_shipdate, c_nationkey@6 as c_nationkey]
------------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_custkey", index: 4 }, Column { name: "c_custkey", index: 0 })]
+--------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@4, c_custkey@0)]
----------------------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "o_custkey", index: 4 }], 4), input_partitions=4
+------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@4], 4), input_partitions=4
--------------------------------------------ProjectionExec: expr=[s_nationkey@0 as s_nationkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, l_shipdate@4 as l_shipdate, o_custkey@6 as o_custkey]
----------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_orderkey", index: 1 }, Column { name: "o_orderkey", index: 0 })]
+------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@1, o_orderkey@0)]
--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 1 }], 4), input_partitions=4
+----------------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4
------------------------------------------------------ProjectionExec: expr=[s_nationkey@1 as s_nationkey, l_orderkey@2 as l_orderkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount, l_shipdate@6 as l_shipdate]
--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_suppkey", index: 0 }, Column { name: "l_suppkey", index: 1 })]
+----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)]
------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4), input_partitions=4
+--------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4
----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false
------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 4), input_partitions=4
+--------------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4
----------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
------------------------------------------------------------------FilterExec: l_shipdate@4 >= 9131 AND l_shipdate@4 <= 9861
--------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
----------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false
--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4), input_partitions=4
+----------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4
------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
--------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl]]}, projection=[o_orderkey, o_custkey], has_header=false
----------------------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4), input_partitions=4
+------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4
--------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
----------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false
------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=4
+--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4
----------------------------------CoalesceBatchesExec: target_batch_size=8192
------------------------------------FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY
--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false
--------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=4
+----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4
------------------------CoalesceBatchesExec: target_batch_size=8192
--------------------------FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE
----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q8.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q8.slt.part
index 38ee2119df..15137267dc 100644
--- a/datafusion/core/tests/sqllogictests/test_files/tpch/q8.slt.part
+++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q8.slt.part
@@ -95,76 +95,76 @@ SortPreservingMergeExec: [o_year@0 ASC NULLS LAST]
----ProjectionExec: expr=[o_year@0 as o_year, CAST(CAST(SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END)@1 AS Decimal128(12, 2)) / CAST(SUM(all_nations.volume)@2 AS Decimal128(12, 2)) AS Decimal128(15, 2)) as mkt_share]
------AggregateExec: mode=FinalPartitioned, gby=[o_year@0 as o_year], aggr=[SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), SUM(all_nations.volume)]
--------CoalesceBatchesExec: target_batch_size=8192
-----------RepartitionExec: partitioning=Hash([Column { name: "o_year", index: 0 }], 4), input_partitions=4
+----------RepartitionExec: partitioning=Hash([o_year@0], 4), input_partitions=4
------------AggregateExec: mode=Partial, gby=[o_year@0 as o_year], aggr=[SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), SUM(all_nations.volume)]
--------------ProjectionExec: expr=[date_part(YEAR, o_orderdate@2) as o_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume, n_name@4 as nation]
----------------CoalesceBatchesExec: target_batch_size=8192
-------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "n_regionkey", index: 3 }, Column { name: "r_regionkey", index: 0 })]
+------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)]
--------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------RepartitionExec: partitioning=Hash([Column { name: "n_regionkey", index: 3 }], 4), input_partitions=4
+----------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4
------------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, o_orderdate@3 as o_orderdate, n_regionkey@4 as n_regionkey, n_name@6 as n_name]
--------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 2 }, Column { name: "n_nationkey", index: 0 })]
+----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)]
------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------RepartitionExec: partitioning=Hash([Column { name: "s_nationkey", index: 2 }], 4), input_partitions=4
+--------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4
----------------------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, s_nationkey@2 as s_nationkey, o_orderdate@3 as o_orderdate, n_regionkey@6 as n_regionkey]
------------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "c_nationkey", index: 4 }, Column { name: "n_nationkey", index: 0 })]
+--------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@4, n_nationkey@0)]
----------------------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "c_nationkey", index: 4 }], 4), input_partitions=4
+------------------------------------------RepartitionExec: partitioning=Hash([c_nationkey@4], 4), input_partitions=4
--------------------------------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, s_nationkey@2 as s_nationkey, o_orderdate@4 as o_orderdate, c_nationkey@6 as c_nationkey]
----------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "o_custkey", index: 3 }, Column { name: "c_custkey", index: 0 })]
+------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@3, c_custkey@0)]
--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "o_custkey", index: 3 }], 4), input_partitions=4
+----------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@3], 4), input_partitions=4
------------------------------------------------------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, s_nationkey@3 as s_nationkey, o_custkey@5 as o_custkey, o_orderdate@6 as o_orderdate]
--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderkey", index: 0 })]
+----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)]
------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4), input_partitions=4
+--------------------------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4
----------------------------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@5 as s_nationkey]
------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_suppkey", index: 1 }, Column { name: "s_suppkey", index: 0 })]
+--------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0)]
----------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "l_suppkey", index: 1 }], 4), input_partitions=4
+------------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4
--------------------------------------------------------------------------ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_suppkey@3 as l_suppkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount]
----------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "p_partkey", index: 0 }, Column { name: "l_partkey", index: 1 })]
+------------------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)]
--------------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------------------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4), input_partitions=4
+----------------------------------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4
------------------------------------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey]
--------------------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
----------------------------------------------------------------------------------------FilterExec: p_type@1 = ECONOMY ANODIZED STEEL
------------------------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
--------------------------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false
--------------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------------------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "l_partkey", index: 1 }], 4), input_partitions=4
+----------------------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4
------------------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
--------------------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount], has_header=false
----------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4), input_partitions=4
+------------------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4
--------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
----------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false
------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4), input_partitions=4
+--------------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4
----------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
------------------------------------------------------------------FilterExec: o_orderdate@2 >= 9131 AND o_orderdate@2 <= 9861
--------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
----------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false
--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "c_custkey", index: 0 }], 4), input_partitions=4
+----------------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4
------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
--------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false
----------------------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=4
+------------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4
--------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
----------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false
------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=4
+--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4
----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false
--------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------RepartitionExec: partitioning=Hash([Column { name: "r_regionkey", index: 0 }], 4), input_partitions=4
+----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4
------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey]
--------------------------CoalesceBatchesExec: target_batch_size=8192
----------------------------FilterExec: r_name@1 = AMERICA
diff --git a/datafusion/core/tests/sqllogictests/test_files/tpch/q9.slt.part b/datafusion/core/tests/sqllogictests/test_files/tpch/q9.slt.part
index b2c49141c5..756b2e2c7c 100644
--- a/datafusion/core/tests/sqllogictests/test_files/tpch/q9.slt.part
+++ b/datafusion/core/tests/sqllogictests/test_files/tpch/q9.slt.part
@@ -82,56 +82,56 @@ GlobalLimitExec: skip=0, fetch=10
------ProjectionExec: expr=[nation@0 as nation, o_year@1 as o_year, SUM(profit.amount)@2 as sum_profit]
--------AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[SUM(profit.amount)]
----------CoalesceBatchesExec: target_batch_size=8192
-------------RepartitionExec: partitioning=Hash([Column { name: "nation", index: 0 }, Column { name: "o_year", index: 1 }], 4), input_partitions=4
+------------RepartitionExec: partitioning=Hash([nation@0, o_year@1], 4), input_partitions=4
--------------AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[SUM(profit.amount)]
----------------ProjectionExec: expr=[n_name@7 as nation, date_part(YEAR, o_orderdate@5) as o_year, l_extendedprice@1 * (Some(1),20,0 - l_discount@2) - ps_supplycost@4 * l_quantity@0 as amount]
------------------CoalesceBatchesExec: target_batch_size=8192
---------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "s_nationkey", index: 3 }, Column { name: "n_nationkey", index: 0 })]
+--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)]
----------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------RepartitionExec: partitioning=Hash([Column { name: "s_nationkey", index: 3 }], 4), input_partitions=4
+------------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4
--------------------------ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@4 as s_nationkey, ps_supplycost@5 as ps_supplycost, o_orderdate@7 as o_orderdate]
----------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_orderkey", index: 0 }, Column { name: "o_orderkey", index: 0 })]
+------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)]
--------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------RepartitionExec: partitioning=Hash([Column { name: "l_orderkey", index: 0 }], 4), input_partitions=4
+----------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4
------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_quantity@3 as l_quantity, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount, s_nationkey@6 as s_nationkey, ps_supplycost@9 as ps_supplycost]
--------------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_suppkey", index: 2 }, Column { name: "ps_suppkey", index: 1 }), (Column { name: "l_partkey", index: 1 }, Column { name: "ps_partkey", index: 0 })]
+----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, ps_suppkey@1), (l_partkey@1, ps_partkey@0)]
------------------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "l_suppkey", index: 2 }, Column { name: "l_partkey", index: 1 }], 4), input_partitions=4
+--------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2, l_partkey@1], 4), input_partitions=4
----------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_partkey@1 as l_partkey, l_suppkey@2 as l_suppkey, l_quantity@3 as l_quantity, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount, s_nationkey@7 as s_nationkey]
------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "l_suppkey", index: 2 }, Column { name: "s_suppkey", index: 0 })]
+--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, s_suppkey@0)]
----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "l_suppkey", index: 2 }], 4), input_partitions=4
+------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2], 4), input_partitions=4
--------------------------------------------------------ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_partkey@2 as l_partkey, l_suppkey@3 as l_suppkey, l_quantity@4 as l_quantity, l_extendedprice@5 as l_extendedprice, l_discount@6 as l_discount]
----------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(Column { name: "p_partkey", index: 0 }, Column { name: "l_partkey", index: 1 })]
+------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)]
--------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "p_partkey", index: 0 }], 4), input_partitions=4
+----------------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4
------------------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey]
--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
----------------------------------------------------------------------FilterExec: p_name@1 LIKE %green%
------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
--------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false
--------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "l_partkey", index: 1 }], 4), input_partitions=4
+----------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4
------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
--------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/lineitem.tbl]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], has_header=false
----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "s_suppkey", index: 0 }], 4), input_partitions=4
+------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4
--------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
----------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false
------------------------------------------CoalesceBatchesExec: target_batch_size=8192
---------------------------------------------RepartitionExec: partitioning=Hash([Column { name: "ps_suppkey", index: 1 }, Column { name: "ps_partkey", index: 0 }], 4), input_partitions=4
+--------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4
----------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/partsupp.tbl]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false
--------------------------------CoalesceBatchesExec: target_batch_size=8192
-----------------------------------RepartitionExec: partitioning=Hash([Column { name: "o_orderkey", index: 0 }], 4), input_partitions=4
+----------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4
------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
--------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/orders.tbl]]}, projection=[o_orderkey, o_orderdate], has_header=false
----------------------CoalesceBatchesExec: target_batch_size=8192
-------------------------RepartitionExec: partitioning=Hash([Column { name: "n_nationkey", index: 0 }], 4), input_partitions=4
+------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4
--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/sqllogictests/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false
diff --git a/datafusion/core/tests/sqllogictests/test_files/union.slt b/datafusion/core/tests/sqllogictests/test_files/union.slt
index 2f33437ca1..94c9eef893 100644
--- a/datafusion/core/tests/sqllogictests/test_files/union.slt
+++ b/datafusion/core/tests/sqllogictests/test_files/union.slt
@@ -204,30 +204,30 @@ physical_plan
UnionExec
--ProjectionExec: expr=[id@0 as id, name@1 as name]
----CoalesceBatchesExec: target_batch_size=8192
-------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(Column { name: "id", index: 0 }, Column { name: "CAST(t2.id AS Int32)", index: 2 }), (Column { name: "name", index: 1 }, Column { name: "name", index: 1 })]
+------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(id@0, CAST(t2.id AS Int32)@2), (name@1, name@1)]
--------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[]
----------CoalesceBatchesExec: target_batch_size=8192
-------------RepartitionExec: partitioning=Hash([Column { name: "id", index: 0 }, Column { name: "name", index: 1 }], 4), input_partitions=4
+------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4
--------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[]
----------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0]
--------CoalesceBatchesExec: target_batch_size=8192
-----------RepartitionExec: partitioning=Hash([Column { name: "CAST(t2.id AS Int32)", index: 2 }, Column { name: "name", index: 1 }], 4), input_partitions=4
+----------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4
------------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)]
--------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0]
--ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name]
----ProjectionExec: expr=[id@0 as id, name@1 as name]
------CoalesceBatchesExec: target_batch_size=8192
---------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(Column { name: "CAST(t2.id AS Int32)", index: 2 }, Column { name: "id", index: 0 }), (Column { name: "name", index: 1 }, Column { name: "name", index: 1 })]
+--------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)]
----------CoalesceBatchesExec: target_batch_size=8192
-------------RepartitionExec: partitioning=Hash([Column { name: "CAST(t2.id AS Int32)", index: 2 }, Column { name: "name", index: 1 }], 4), input_partitions=4
+------------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4
--------------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)]
----------------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[]
------------------CoalesceBatchesExec: target_batch_size=8192
---------------------RepartitionExec: partitioning=Hash([Column { name: "id", index: 0 }, Column { name: "name", index: 1 }], 4), input_partitions=4
+--------------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4
----------------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[]
------------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0]
----------CoalesceBatchesExec: target_batch_size=8192
-------------RepartitionExec: partitioning=Hash([Column { name: "id", index: 0 }, Column { name: "name", index: 1 }], 4), input_partitions=4
+------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4
--------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0]
query IT rowsort
@@ -274,24 +274,24 @@ Union
physical_plan
InterleaveExec
--CoalesceBatchesExec: target_batch_size=8192
-----HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(Column { name: "name", index: 0 }, Column { name: "name", index: 0 })]
+----HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(name@0, name@0)]
------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[]
--------CoalesceBatchesExec: target_batch_size=8192
-----------RepartitionExec: partitioning=Hash([Column { name: "name", index: 0 }], 4), input_partitions=4
+----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4
------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[]
--------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0]
------CoalesceBatchesExec: target_batch_size=8192
---------RepartitionExec: partitioning=Hash([Column { name: "name", index: 0 }], 4), input_partitions=4
+--------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4
----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0]
--CoalesceBatchesExec: target_batch_size=8192
-----HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(Column { name: "name", index: 0 }, Column { name: "name", index: 0 })]
+----HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(name@0, name@0)]
------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[]
--------CoalesceBatchesExec: target_batch_size=8192
-----------RepartitionExec: partitioning=Hash([Column { name: "name", index: 0 }], 4), input_partitions=4
+----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4
------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[]
--------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0]
------CoalesceBatchesExec: target_batch_size=8192
---------RepartitionExec: partitioning=Hash([Column { name: "name", index: 0 }], 4), input_partitions=4
+--------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4
----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0]
# union_upcast_types
@@ -351,12 +351,12 @@ ProjectionExec: expr=[COUNT(UInt8(1))@1 as COUNT(UInt8(1))]
----InterleaveExec
------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[]
--------CoalesceBatchesExec: target_batch_size=8192
-----------RepartitionExec: partitioning=Hash([Column { name: "name", index: 0 }], 4), input_partitions=4
+----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4
------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[]
--------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0]
------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[]
--------CoalesceBatchesExec: target_batch_size=8192
-----------RepartitionExec: partitioning=Hash([Column { name: "name", index: 0 }], 4), input_partitions=4
+----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4
------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[]
--------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0]
@@ -465,7 +465,7 @@ UnionExec
--ProjectionExec: expr=[Int64(1)@0 as a]
----AggregateExec: mode=FinalPartitioned, gby=[Int64(1)@0 as Int64(1)], aggr=[]
------CoalesceBatchesExec: target_batch_size=8192
---------RepartitionExec: partitioning=Hash([Column { name: "Int64(1)", index: 0 }], 4), input_partitions=4
+--------RepartitionExec: partitioning=Hash([Int64(1)@0], 4), input_partitions=4
----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
------------AggregateExec: mode=Partial, gby=[1 as Int64(1)], aggr=[]
--------------EmptyExec: produce_one_row=true
@@ -497,7 +497,7 @@ UnionExec
--ProjectionExec: expr=[COUNT(UInt8(1))@1 as count, n@0 as n]
----AggregateExec: mode=FinalPartitioned, gby=[n@0 as n], aggr=[COUNT(UInt8(1))]
------CoalesceBatchesExec: target_batch_size=8192
---------RepartitionExec: partitioning=Hash([Column { name: "n", index: 0 }], 4), input_partitions=4
+--------RepartitionExec: partitioning=Hash([n@0], 4), input_partitions=4
----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1
------------AggregateExec: mode=Partial, gby=[n@0 as n], aggr=[COUNT(UInt8(1))]
--------------ProjectionExec: expr=[5 as n]
diff --git a/datafusion/core/tests/sqllogictests/test_files/window.slt b/datafusion/core/tests/sqllogictests/test_files/window.slt
index c0b861fd8a..08d1a5616e 100644
--- a/datafusion/core/tests/sqllogictests/test_files/window.slt
+++ b/datafusion/core/tests/sqllogictests/test_files/window.slt
@@ -275,7 +275,7 @@ SortPreservingMergeExec: [b@0 ASC NULLS LAST]
----ProjectionExec: expr=[b@0 as b, MAX(d.a)@1 as max_a]
------AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[MAX(d.a)]
--------CoalesceBatchesExec: target_batch_size=8192
-----------RepartitionExec: partitioning=Hash([Column { name: "b", index: 0 }], 4), input_partitions=4
+----------RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4
------------AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[MAX(d.a)]
--------------UnionExec
----------------ProjectionExec: expr=[1 as a, aa as b]
@@ -362,7 +362,7 @@ SortPreservingMergeExec: [b@0 ASC NULLS LAST]
--------BoundedWindowAggExec: wdw=[ROW_NUMBER(): Ok(Field { name: "ROW_NUMBER()", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow }], mode=[Sorted]
----------SortExec: expr=[b@1 ASC NULLS LAST,a@0 ASC NULLS LAST]
------------CoalesceBatchesExec: target_batch_size=8192
---------------RepartitionExec: partitioning=Hash([Column { name: "b", index: 1 }], 4), input_partitions=4
+--------------RepartitionExec: partitioning=Hash([b@1], 4), input_partitions=4
----------------UnionExec
------------------ProjectionExec: expr=[1 as a, aa as b]
--------------------EmptyExec: produce_one_row=true
@@ -1286,12 +1286,12 @@ ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_10
--BoundedWindowAggExec: wdw=[COUNT(UInt8(1)): Ok(Field { name: "COUNT(UInt8(1))", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }], mode=[Sorted]
----SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST]
------CoalesceBatchesExec: target_batch_size=4096
---------RepartitionExec: partitioning=Hash([Column { name: "c1", index: 0 }], 2), input_partitions=2
+--------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2
----------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as SUM(aggregate_test_100.c4)]
------------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4): Ok(Field { name: "SUM(aggregate_test_100.c4)", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }], mode=[Sorted]
--------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST]
----------------CoalesceBatchesExec: target_batch_size=4096
-------------------RepartitionExec: partitioning=Hash([Column { name: "c1", index: 0 }, Column { name: "c2", index: 1 }], 2), input_partitions=2
+------------------RepartitionExec: partitioning=Hash([c1@0, c2@1], 2), input_partitions=2
--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c4], has_header=true
@@ -1750,7 +1750,7 @@ ProjectionExec: expr=[COUNT(UInt8(1))@0 as global_count]
----------ProjectionExec: expr=[c1@0 as c1]
------------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[COUNT(UInt8(1))]
--------------CoalesceBatchesExec: target_batch_size=4096
-----------------RepartitionExec: partitioning=Hash([Column { name: "c1", index: 0 }], 2), input_partitions=2
+----------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2
------------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[COUNT(UInt8(1))]
--------------------ProjectionExec: expr=[c1@0 as c1]
----------------------CoalesceBatchesExec: target_batch_size=4096
@@ -1797,7 +1797,7 @@ GlobalLimitExec: skip=0, fetch=5
------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: "SUM(aggregate_test_100.c9)", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow }], mode=[Sorted]
--------SortExec: expr=[c3@0 ASC NULLS LAST,c9@1 DESC]
----------CoalesceBatchesExec: target_batch_size=4096
-------------RepartitionExec: partitioning=Hash([Column { name: "c3", index: 0 }], 2), input_partitions=2
+------------RepartitionExec: partitioning=Hash([c3@0], 2), input_partitions=2
--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
----------------ProjectionExec: expr=[c3@1 as c3, c9@2 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as SUM(aggregate_test_100.c9)]
------------------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: "SUM(aggregate_test_100.c9)", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int16(NULL)), end_bound: CurrentRow }], mode=[Sorted]
@@ -1839,7 +1839,7 @@ SortPreservingMergeExec: [c1@0 ASC NULLS LAST]
----BoundedWindowAggExec: wdw=[ROW_NUMBER(): Ok(Field { name: "ROW_NUMBER()", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }], mode=[Sorted]
------SortExec: expr=[c1@0 ASC NULLS LAST]
--------CoalesceBatchesExec: target_batch_size=4096
-----------RepartitionExec: partitioning=Hash([Column { name: "c1", index: 0 }], 2), input_partitions=2
+----------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2
------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true
@@ -1966,7 +1966,7 @@ SortExec: expr=[c1@0 ASC NULLS LAST]
------BoundedWindowAggExec: wdw=[ROW_NUMBER(): Ok(Field { name: "ROW_NUMBER()", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }], mode=[Sorted]
--------SortExec: expr=[c1@0 ASC NULLS LAST]
----------CoalesceBatchesExec: target_batch_size=4096
-------------RepartitionExec: partitioning=Hash([Column { name: "c1", index: 0 }], 2), input_partitions=2
+------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2
--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true
@@ -1995,7 +1995,7 @@ SortExec: expr=[c1@0 ASC NULLS LAST]
----------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: "SUM(aggregate_test_100.c9)", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(3)) }], mode=[Sorted]
------------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 ASC NULLS LAST]
--------------CoalesceBatchesExec: target_batch_size=4096
-----------------RepartitionExec: partitioning=Hash([Column { name: "c1", index: 0 }], 2), input_partitions=2
+----------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2
------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1
--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true
diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs
index a0926b7d31..9c3b53906a 100644
--- a/datafusion/expr/src/expr.rs
+++ b/datafusion/expr/src/expr.rs
@@ -79,7 +79,7 @@ use std::sync::Arc;
/// assert_eq!(binary_expr.op, Operator::Eq);
/// }
/// ```
-#[derive(Clone, PartialEq, Eq, Hash)]
+#[derive(Clone, PartialEq, Eq, Hash, Debug)]
pub enum Expr {
/// An expression with a specific name.
Alias(Box<Expr>, String),
@@ -181,7 +181,7 @@ pub enum Expr {
}
/// Binary expression
-#[derive(Clone, PartialEq, Eq, Hash)]
+#[derive(Clone, PartialEq, Eq, Hash, Debug)]
pub struct BinaryExpr {
/// Left-hand side of the expression
pub left: Box<Expr>,
@@ -258,7 +258,7 @@ impl Case {
}
/// LIKE expression
-#[derive(Clone, PartialEq, Eq, Hash)]
+#[derive(Clone, PartialEq, Eq, Hash, Debug)]
pub struct Like {
pub negated: bool,
pub expr: Box<Expr>,
@@ -284,7 +284,7 @@ impl Like {
}
/// BETWEEN expression
-#[derive(Clone, PartialEq, Eq, Hash)]
+#[derive(Clone, PartialEq, Eq, Hash, Debug)]
pub struct Between {
/// The value to compare
pub expr: Box<Expr>,
@@ -309,7 +309,7 @@ impl Between {
}
/// ScalarFunction expression
-#[derive(Clone, PartialEq, Eq, Hash)]
+#[derive(Clone, PartialEq, Eq, Hash, Debug)]
pub struct ScalarFunction {
/// The function
pub fun: built_in_function::BuiltinScalarFunction,
@@ -325,7 +325,7 @@ impl ScalarFunction {
}
/// ScalarUDF expression
-#[derive(Clone, PartialEq, Eq, Hash)]
+#[derive(Clone, PartialEq, Eq, Hash, Debug)]
pub struct ScalarUDF {
/// The function
pub fun: Arc<crate::ScalarUDF>,
@@ -341,7 +341,7 @@ impl ScalarUDF {
}
/// Returns the field of a [`arrow::array::ListArray`] or [`arrow::array::StructArray`] by key
-#[derive(Clone, PartialEq, Eq, Hash)]
+#[derive(Clone, PartialEq, Eq, Hash, Debug)]
pub struct GetIndexedField {
/// the expression to take the field from
pub expr: Box<Expr>,
@@ -357,7 +357,7 @@ impl GetIndexedField {
}
/// Cast expression
-#[derive(Clone, PartialEq, Eq, Hash)]
+#[derive(Clone, PartialEq, Eq, Hash, Debug)]
pub struct Cast {
/// The expression being cast
pub expr: Box<Expr>,
@@ -373,7 +373,7 @@ impl Cast {
}
/// TryCast Expression
-#[derive(Clone, PartialEq, Eq, Hash)]
+#[derive(Clone, PartialEq, Eq, Hash, Debug)]
pub struct TryCast {
/// The expression being cast
pub expr: Box<Expr>,
@@ -389,7 +389,7 @@ impl TryCast {
}
/// SORT expression
-#[derive(Clone, PartialEq, Eq, Hash)]
+#[derive(Clone, PartialEq, Eq, Hash, Debug)]
pub struct Sort {
/// The expression to sort on
pub expr: Box<Expr>,
@@ -411,7 +411,7 @@ impl Sort {
}
/// Aggregate function
-#[derive(Clone, PartialEq, Eq, Hash)]
+#[derive(Clone, PartialEq, Eq, Hash, Debug)]
pub struct AggregateFunction {
/// Name of the function
pub fun: aggregate_function::AggregateFunction,
@@ -444,7 +444,7 @@ impl AggregateFunction {
}
/// Window function
-#[derive(Clone, PartialEq, Eq, Hash)]
+#[derive(Clone, PartialEq, Eq, Hash, Debug)]
pub struct WindowFunction {
/// Name of the function
pub fun: window_function::WindowFunction,
@@ -478,7 +478,7 @@ impl WindowFunction {
}
// Exists expression.
-#[derive(Clone, PartialEq, Eq, Hash)]
+#[derive(Clone, PartialEq, Eq, Hash, Debug)]
pub struct Exists {
/// subquery that will produce a single column of data
pub subquery: Subquery,
@@ -493,7 +493,7 @@ impl Exists {
}
}
-#[derive(Clone, PartialEq, Eq, Hash)]
+#[derive(Clone, PartialEq, Eq, Hash, Debug)]
pub struct AggregateUDF {
/// The function
pub fun: Arc<udaf::AggregateUDF>,
@@ -523,7 +523,7 @@ impl AggregateUDF {
}
/// InList expression
-#[derive(Clone, PartialEq, Eq, Hash)]
+#[derive(Clone, PartialEq, Eq, Hash, Debug)]
pub struct InList {
/// The expression to compare
pub expr: Box<Expr>,
@@ -545,7 +545,7 @@ impl InList {
}
/// IN subquery
-#[derive(Clone, PartialEq, Eq, Hash)]
+#[derive(Clone, PartialEq, Eq, Hash, Debug)]
pub struct InSubquery {
/// The expression to compare
pub expr: Box<Expr>,
@@ -567,7 +567,7 @@ impl InSubquery {
}
/// Placeholder
-#[derive(Clone, PartialEq, Eq, Hash)]
+#[derive(Clone, PartialEq, Eq, Hash, Debug)]
pub struct Placeholder {
/// The identifier of the parameter (e.g, $1 or $foo)
pub id: String,
@@ -587,7 +587,7 @@ impl Placeholder {
/// for Postgres definition.
/// See <https://spark.apache.org/docs/latest/sql-ref-syntax-qry-select-groupby.html>
/// for Apache Spark definition.
-#[derive(Clone, PartialEq, Eq, Hash)]
+#[derive(Clone, PartialEq, Eq, Hash, Debug)]
pub enum GroupingSet {
/// Rollup grouping sets
Rollup(Vec<Expr>),
@@ -898,20 +898,23 @@ impl Expr {
}
}
-/// Format expressions for display as part of a logical plan. In many cases, this will produce
-/// similar output to `Expr.name()` except that column names will be prefixed with '#'.
-impl fmt::Display for Expr {
- fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
- write!(f, "{self:?}")
- }
+#[macro_export]
+macro_rules! expr_vec_fmt {
+ ( $ARRAY:expr ) => {{
+ $ARRAY
+ .iter()
+ .map(|e| format!("{e}"))
+ .collect::<Vec<String>>()
+ .join(", ")
+ }};
}
/// Format expressions for display as part of a logical plan. In many cases, this will produce
/// similar output to `Expr.name()` except that column names will be prefixed with '#'.
-impl fmt::Debug for Expr {
+impl fmt::Display for Expr {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
match self {
- Expr::Alias(expr, alias) => write!(f, "{expr:?} AS {alias}"),
+ Expr::Alias(expr, alias) => write!(f, "{expr} AS {alias}"),
Expr::Column(c) => write!(f, "{c}"),
Expr::OuterReferenceColumn(_, c) => write!(f, "outer_ref({c})"),
Expr::ScalarVariable(_, var_names) => write!(f, "{}", var_names.join(".")),
@@ -919,32 +922,32 @@ impl fmt::Debug for Expr {
Expr::Case(case) => {
write!(f, "CASE ")?;
if let Some(e) = &case.expr {
- write!(f, "{e:?} ")?;
+ write!(f, "{e} ")?;
}
for (w, t) in &case.when_then_expr {
- write!(f, "WHEN {w:?} THEN {t:?} ")?;
+ write!(f, "WHEN {w} THEN {t} ")?;
}
if let Some(e) = &case.else_expr {
- write!(f, "ELSE {e:?} ")?;
+ write!(f, "ELSE {e} ")?;
}
write!(f, "END")
}
Expr::Cast(Cast { expr, data_type }) => {
- write!(f, "CAST({expr:?} AS {data_type:?})")
+ write!(f, "CAST({expr} AS {data_type:?})")
}
Expr::TryCast(TryCast { expr, data_type }) => {
- write!(f, "TRY_CAST({expr:?} AS {data_type:?})")
+ write!(f, "TRY_CAST({expr} AS {data_type:?})")
}
- Expr::Not(expr) => write!(f, "NOT {expr:?}"),
- Expr::Negative(expr) => write!(f, "(- {expr:?})"),
- Expr::IsNull(expr) => write!(f, "{expr:?} IS NULL"),
- Expr::IsNotNull(expr) => write!(f, "{expr:?} IS NOT NULL"),
- Expr::IsTrue(expr) => write!(f, "{expr:?} IS TRUE"),
- Expr::IsFalse(expr) => write!(f, "{expr:?} IS FALSE"),
- Expr::IsUnknown(expr) => write!(f, "{expr:?} IS UNKNOWN"),
- Expr::IsNotTrue(expr) => write!(f, "{expr:?} IS NOT TRUE"),
- Expr::IsNotFalse(expr) => write!(f, "{expr:?} IS NOT FALSE"),
- Expr::IsNotUnknown(expr) => write!(f, "{expr:?} IS NOT UNKNOWN"),
+ Expr::Not(expr) => write!(f, "NOT {expr}"),
+ Expr::Negative(expr) => write!(f, "(- {expr})"),
+ Expr::IsNull(expr) => write!(f, "{expr} IS NULL"),
+ Expr::IsNotNull(expr) => write!(f, "{expr} IS NOT NULL"),
+ Expr::IsTrue(expr) => write!(f, "{expr} IS TRUE"),
+ Expr::IsFalse(expr) => write!(f, "{expr} IS FALSE"),
+ Expr::IsUnknown(expr) => write!(f, "{expr} IS UNKNOWN"),
+ Expr::IsNotTrue(expr) => write!(f, "{expr} IS NOT TRUE"),
+ Expr::IsNotFalse(expr) => write!(f, "{expr} IS NOT FALSE"),
+ Expr::IsNotUnknown(expr) => write!(f, "{expr} IS NOT UNKNOWN"),
Expr::Exists(Exists {
subquery,
negated: true,
@@ -957,12 +960,12 @@ impl fmt::Debug for Expr {
expr,
subquery,
negated: true,
- }) => write!(f, "{expr:?} NOT IN ({subquery:?})"),
+ }) => write!(f, "{expr} NOT IN ({subquery:?})"),
Expr::InSubquery(InSubquery {
expr,
subquery,
negated: false,
- }) => write!(f, "{expr:?} IN ({subquery:?})"),
+ }) => write!(f, "{expr} IN ({subquery:?})"),
Expr::ScalarSubquery(subquery) => write!(f, "({subquery:?})"),
Expr::BinaryExpr(expr) => write!(f, "{expr}"),
Expr::Sort(Sort {
@@ -971,9 +974,9 @@ impl fmt::Debug for Expr {
nulls_first,
}) => {
if *asc {
- write!(f, "{expr:?} ASC")?;
+ write!(f, "{expr} ASC")?;
} else {
- write!(f, "{expr:?} DESC")?;
+ write!(f, "{expr} DESC")?;
}
if *nulls_first {
write!(f, " NULLS FIRST")
@@ -982,10 +985,10 @@ impl fmt::Debug for Expr {
}
}
Expr::ScalarFunction(func) => {
- fmt_function(f, &func.fun.to_string(), false, &func.args, false)
+ fmt_function(f, &func.fun.to_string(), false, &func.args, true)
}
Expr::ScalarUDF(ScalarUDF { fun, args }) => {
- fmt_function(f, &fun.name, false, args, false)
+ fmt_function(f, &fun.name, false, args, true)
}
Expr::WindowFunction(WindowFunction {
fun,
@@ -994,12 +997,12 @@ impl fmt::Debug for Expr {
order_by,
window_frame,
}) => {
- fmt_function(f, &fun.to_string(), false, args, false)?;
+ fmt_function(f, &fun.to_string(), false, args, true)?;
if !partition_by.is_empty() {
- write!(f, " PARTITION BY {partition_by:?}")?;
+ write!(f, " PARTITION BY [{}]", expr_vec_fmt!(partition_by))?;
}
if !order_by.is_empty() {
- write!(f, " ORDER BY {order_by:?}")?;
+ write!(f, " ORDER BY [{}]", expr_vec_fmt!(order_by))?;
}
write!(
f,
@@ -1021,7 +1024,7 @@ impl fmt::Debug for Expr {
write!(f, " FILTER (WHERE {fe})")?;
}
if let Some(ob) = order_by {
- write!(f, " ORDER BY {ob:?}")?;
+ write!(f, " ORDER BY [{}]", expr_vec_fmt!(ob))?;
}
Ok(())
}
@@ -1032,12 +1035,12 @@ impl fmt::Debug for Expr {
order_by,
..
}) => {
- fmt_function(f, &fun.name, false, args, false)?;
+ fmt_function(f, &fun.name, false, args, true)?;
if let Some(fe) = filter {
write!(f, " FILTER (WHERE {fe})")?;
}
if let Some(ob) = order_by {
- write!(f, " ORDER BY {ob:?}")?;
+ write!(f, " ORDER BY [{}]", expr_vec_fmt!(ob))?;
}
Ok(())
}
@@ -1048,9 +1051,9 @@ impl fmt::Debug for Expr {
high,
}) => {
if *negated {
- write!(f, "{expr:?} NOT BETWEEN {low:?} AND {high:?}")
+ write!(f, "{expr} NOT BETWEEN {low} AND {high}")
} else {
- write!(f, "{expr:?} BETWEEN {low:?} AND {high:?}")
+ write!(f, "{expr} BETWEEN {low} AND {high}")
}
}
Expr::Like(Like {
@@ -1059,14 +1062,14 @@ impl fmt::Debug for Expr {
pattern,
escape_char,
}) => {
- write!(f, "{expr:?}")?;
+ write!(f, "{expr}")?;
if *negated {
write!(f, " NOT")?;
}
if let Some(char) = escape_char {
- write!(f, " LIKE {pattern:?} ESCAPE '{char}'")
+ write!(f, " LIKE {pattern} ESCAPE '{char}'")
} else {
- write!(f, " LIKE {pattern:?}")
+ write!(f, " LIKE {pattern}")
}
}
Expr::ILike(Like {
@@ -1075,14 +1078,14 @@ impl fmt::Debug for Expr {
pattern,
escape_char,
}) => {
- write!(f, "{expr:?}")?;
+ write!(f, "{expr}")?;
if *negated {
write!(f, " NOT")?;
}
if let Some(char) = escape_char {
- write!(f, " ILIKE {pattern:?} ESCAPE '{char}'")
+ write!(f, " ILIKE {pattern} ESCAPE '{char}'")
} else {
- write!(f, " ILIKE {pattern:?}")
+ write!(f, " ILIKE {pattern}")
}
}
Expr::SimilarTo(Like {
@@ -1091,14 +1094,14 @@ impl fmt::Debug for Expr {
pattern,
escape_char,
}) => {
- write!(f, "{expr:?}")?;
+ write!(f, "{expr}")?;
if *negated {
write!(f, " NOT")?;
}
if let Some(char) = escape_char {
- write!(f, " SIMILAR TO {pattern:?} ESCAPE '{char}'")
+ write!(f, " SIMILAR TO {pattern} ESCAPE '{char}'")
} else {
- write!(f, " SIMILAR TO {pattern:?}")
+ write!(f, " SIMILAR TO {pattern}")
}
}
Expr::InList(InList {
@@ -1107,40 +1110,24 @@ impl fmt::Debug for Expr {
negated,
}) => {
if *negated {
- write!(f, "{expr:?} NOT IN ({list:?})")
+ write!(f, "{expr} NOT IN ([{}])", expr_vec_fmt!(list))
} else {
- write!(f, "{expr:?} IN ({list:?})")
+ write!(f, "{expr} IN ([{}])", expr_vec_fmt!(list))
}
}
Expr::Wildcard => write!(f, "*"),
Expr::QualifiedWildcard { qualifier } => write!(f, "{qualifier}.*"),
Expr::GetIndexedField(GetIndexedField { key, expr }) => {
- write!(f, "({expr:?})[{key}]")
+ write!(f, "({expr})[{key}]")
}
Expr::GroupingSet(grouping_sets) => match grouping_sets {
GroupingSet::Rollup(exprs) => {
// ROLLUP (c0, c1, c2)
- write!(
- f,
- "ROLLUP ({})",
- exprs
- .iter()
- .map(|e| format!("{e}"))
- .collect::<Vec<String>>()
- .join(", ")
- )
+ write!(f, "ROLLUP ({})", expr_vec_fmt!(exprs))
}
GroupingSet::Cube(exprs) => {
// CUBE (c0, c1, c2)
- write!(
- f,
- "CUBE ({})",
- exprs
- .iter()
- .map(|e| format!("{e}"))
- .collect::<Vec<String>>()
- .join(", ")
- )
+ write!(f, "CUBE ({})", expr_vec_fmt!(exprs))
}
GroupingSet::GroupingSets(lists_of_exprs) => {
// GROUPING SETS ((c0), (c1, c2), (c3, c4))
@@ -1149,14 +1136,7 @@ impl fmt::Debug for Expr {
"GROUPING SETS ({})",
lists_of_exprs
.iter()
- .map(|exprs| format!(
- "({})",
- exprs
- .iter()
- .map(|e| format!("{e}"))
- .collect::<Vec<String>>()
- .join(", ")
- ))
+ .map(|exprs| format!("({})", expr_vec_fmt!(exprs)))
.collect::<Vec<String>>()
.join(", ")
)
@@ -1364,10 +1344,10 @@ fn create_name(e: &Expr) -> Result<String> {
let mut parts: Vec<String> =
vec![create_function_name(&fun.to_string(), false, args)?];
if !partition_by.is_empty() {
- parts.push(format!("PARTITION BY {partition_by:?}"));
+ parts.push(format!("PARTITION BY [{}]", expr_vec_fmt!(partition_by)));
}
if !order_by.is_empty() {
- parts.push(format!("ORDER BY {order_by:?}"));
+ parts.push(format!("ORDER BY [{}]", expr_vec_fmt!(order_by)));
}
parts.push(format!("{window_frame}"));
Ok(parts.join(" "))
@@ -1384,7 +1364,7 @@ fn create_name(e: &Expr) -> Result<String> {
name = format!("{name} FILTER (WHERE {fe})");
};
if let Some(order_by) = order_by {
- name = format!("{name} ORDER BY {order_by:?}");
+ name = format!("{name} ORDER BY [{}]", expr_vec_fmt!(order_by));
};
Ok(name)
}
@@ -1403,7 +1383,7 @@ fn create_name(e: &Expr) -> Result<String> {
info += &format!(" FILTER (WHERE {fe})");
}
if let Some(ob) = order_by {
- info += &format!(" ORDER BY ({ob:?})");
+ info += &format!(" ORDER BY ([{}])", expr_vec_fmt!(ob));
}
Ok(format!("{}({}){}", fun.name, names.join(","), info))
}
@@ -1488,7 +1468,6 @@ mod test {
let expected = "CASE a WHEN Int32(1) THEN Boolean(true) WHEN Int32(0) THEN Boolean(false) ELSE NULL END";
assert_eq!(expected, expr.canonical_name());
assert_eq!(expected, format!("{expr}"));
- assert_eq!(expected, format!("{expr:?}"));
assert_eq!(expected, expr.display_name()?);
Ok(())
}
@@ -1502,7 +1481,6 @@ mod test {
let expected_canonical = "CAST(Float32(1.23) AS Utf8)";
assert_eq!(expected_canonical, expr.canonical_name());
assert_eq!(expected_canonical, format!("{expr}"));
- assert_eq!(expected_canonical, format!("{expr:?}"));
// note that CAST intentionally has a name that is different from its `Display`
// representation. CAST does not change the name of expressions.
assert_eq!("Float32(1.23)", expr.display_name()?);
diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs
index 819f8e4aa7..64d0280203 100644
--- a/datafusion/expr/src/expr_fn.rs
+++ b/datafusion/expr/src/expr_fn.rs
@@ -814,9 +814,9 @@ mod test {
fn filter_is_null_and_is_not_null() {
let col_null = col("col1");
let col_not_null = ident("col2");
- assert_eq!(format!("{:?}", col_null.is_null()), "col1 IS NULL");
+ assert_eq!(format!("{}", col_null.is_null()), "col1 IS NULL");
assert_eq!(
- format!("{:?}", col_not_null.is_not_null()),
+ format!("{}", col_not_null.is_not_null()),
"col2 IS NOT NULL"
);
}
diff --git a/datafusion/expr/src/expr_rewriter/mod.rs b/datafusion/expr/src/expr_rewriter/mod.rs
index bc08bb4d66..cbd336d934 100644
--- a/datafusion/expr/src/expr_rewriter/mod.rs
+++ b/datafusion/expr/src/expr_rewriter/mod.rs
@@ -275,12 +275,12 @@ mod test {
type N = Expr;
fn pre_visit(&mut self, expr: &Expr) -> Result<RewriteRecursion> {
- self.v.push(format!("Previsited {expr:?}"));
+ self.v.push(format!("Previsited {expr}"));
Ok(RewriteRecursion::Continue)
}
fn mutate(&mut self, expr: Expr) -> Result<Expr> {
- self.v.push(format!("Mutated {expr:?}"));
+ self.v.push(format!("Mutated {expr}"));
Ok(expr)
}
}
diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs
index 08a167ff0d..3d34c087ac 100644
--- a/datafusion/expr/src/logical_plan/builder.rs
+++ b/datafusion/expr/src/logical_plan/builder.rs
@@ -1113,7 +1113,7 @@ pub(crate) fn validate_unique_names<'a>(
Some((existing_position, existing_expr)) => {
Err(DataFusionError::Plan(
format!("{node_name} require unique expression names \
- but the expression \"{existing_expr:?}\" at position {existing_position} and \"{expr:?}\" \
+ but the expression \"{existing_expr}\" at position {existing_position} and \"{expr}\" \
at position {position} have the same name. Consider aliasing (\"AS\") one of them.",
)
))
@@ -1315,7 +1315,7 @@ pub fn wrap_projection_for_join_if_necessary(
// then a and cast(a as int) will use the same field name - `a` in projection schema.
// https://github.com/apache/arrow-datafusion/issues/4478
if matches!(key, Expr::Cast(_)) || matches!(key, Expr::TryCast(_)) {
- let alias = format!("{key:?}");
+ let alias = format!("{key}");
key.clone().alias(alias)
} else {
key.clone()
diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs
index 4f21ad38c4..fad52fc3a3 100644
--- a/datafusion/expr/src/logical_plan/plan.rs
+++ b/datafusion/expr/src/logical_plan/plan.rs
@@ -17,9 +17,9 @@
//! Logical plan types
-use crate::expr::InSubquery;
-use crate::expr::{Exists, Placeholder};
+use crate::expr::{Exists, InSubquery, Placeholder};
use crate::expr_rewriter::create_col_from_scalar_expr;
+use crate::expr_vec_fmt;
use crate::logical_plan::display::{GraphvizVisitor, IndentVisitor};
use crate::logical_plan::extension::UserDefinedLogicalNode;
use crate::logical_plan::{DmlStatement, Statement};
@@ -1018,15 +1018,24 @@ impl LogicalPlan {
}
if !full_filter.is_empty() {
- write!(f, ", full_filters={full_filter:?}")?;
+ write!(
+ f,
+ ", full_filters=[{}]",
+ expr_vec_fmt!(full_filter)
+ )?;
};
if !partial_filter.is_empty() {
- write!(f, ", partial_filters={partial_filter:?}")?;
+ write!(
+ f,
+ ", partial_filters=[{}]",
+ expr_vec_fmt!(partial_filter)
+ )?;
}
if !unsupported_filters.is_empty() {
write!(
f,
- ", unsupported_filters={unsupported_filters:?}"
+ ", unsupported_filters=[{}]",
+ expr_vec_fmt!(unsupported_filters)
)?;
}
}
@@ -1043,7 +1052,7 @@ impl LogicalPlan {
if i > 0 {
write!(f, ", ")?;
}
- write!(f, "{expr_item:?}")?;
+ write!(f, "{expr_item}")?;
}
Ok(())
}
@@ -1056,11 +1065,15 @@ impl LogicalPlan {
LogicalPlan::Filter(Filter {
predicate: ref expr,
..
- }) => write!(f, "Filter: {expr:?}"),
+ }) => write!(f, "Filter: {expr}"),
LogicalPlan::Window(Window {
ref window_expr, ..
}) => {
- write!(f, "WindowAggr: windowExpr=[{window_expr:?}]")
+ write!(
+ f,
+ "WindowAggr: windowExpr=[[{}]]",
+ expr_vec_fmt!(window_expr)
+ )
}
LogicalPlan::Aggregate(Aggregate {
ref group_expr,
@@ -1068,7 +1081,9 @@ impl LogicalPlan {
..
}) => write!(
f,
- "Aggregate: groupBy=[{group_expr:?}], aggr=[{aggr_expr:?}]"
+ "Aggregate: groupBy=[[{}]], aggr=[[{}]]",
+ expr_vec_fmt!(group_expr),
+ expr_vec_fmt!(aggr_expr)
),
LogicalPlan::Sort(Sort { expr, fetch, .. }) => {
write!(f, "Sort: ")?;
@@ -1076,7 +1091,7 @@ impl LogicalPlan {
if i > 0 {
write!(f, ", ")?;
}
- write!(f, "{expr_item:?}")?;
+ write!(f, "{expr_item}")?;
}
if let Some(a) = fetch {
write!(f, ", fetch={a}")?;
@@ -1130,7 +1145,7 @@ impl LogicalPlan {
}
Partitioning::Hash(expr, n) => {
let hash_expr: Vec<String> =
- expr.iter().map(|e| format!("{e:?}")).collect();
+ expr.iter().map(|e| format!("{e}")).collect();
write!(
f,
"Repartition: Hash({}) partition_count={}",
@@ -1140,7 +1155,7 @@ impl LogicalPlan {
}
Partitioning::DistributeBy(expr) => {
let dist_by_expr: Vec<String> =
- expr.iter().map(|e| format!("{e:?}")).collect();
+ expr.iter().map(|e| format!("{e}")).collect();
write!(
f,
"Repartition: DistributeBy({})",
diff --git a/datafusion/expr/src/operator.rs b/datafusion/expr/src/operator.rs
index 686a681e36..2f14d51797 100644
--- a/datafusion/expr/src/operator.rs
+++ b/datafusion/expr/src/operator.rs
@@ -387,57 +387,57 @@ mod tests {
fn test_operators() {
// Add
assert_eq!(
- format!("{:?}", lit(1u32) + lit(2u32)),
+ format!("{}", lit(1u32) + lit(2u32)),
"UInt32(1) + UInt32(2)"
);
// Sub
assert_eq!(
- format!("{:?}", lit(1u32) - lit(2u32)),
+ format!("{}", lit(1u32) - lit(2u32)),
"UInt32(1) - UInt32(2)"
);
// Mul
assert_eq!(
- format!("{:?}", lit(1u32) * lit(2u32)),
+ format!("{}", lit(1u32) * lit(2u32)),
"UInt32(1) * UInt32(2)"
);
// Div
assert_eq!(
- format!("{:?}", lit(1u32) / lit(2u32)),
+ format!("{}", lit(1u32) / lit(2u32)),
"UInt32(1) / UInt32(2)"
);
// Rem
assert_eq!(
- format!("{:?}", lit(1u32) % lit(2u32)),
+ format!("{}", lit(1u32) % lit(2u32)),
"UInt32(1) % UInt32(2)"
);
// BitAnd
assert_eq!(
- format!("{:?}", lit(1u32) & lit(2u32)),
+ format!("{}", lit(1u32) & lit(2u32)),
"UInt32(1) & UInt32(2)"
);
// BitOr
assert_eq!(
- format!("{:?}", lit(1u32) | lit(2u32)),
+ format!("{}", lit(1u32) | lit(2u32)),
"UInt32(1) | UInt32(2)"
);
// BitXor
assert_eq!(
- format!("{:?}", lit(1u32) ^ lit(2u32)),
+ format!("{}", lit(1u32) ^ lit(2u32)),
"UInt32(1) # UInt32(2)"
);
// Shl
assert_eq!(
- format!("{:?}", lit(1u32) << lit(2u32)),
+ format!("{}", lit(1u32) << lit(2u32)),
"UInt32(1) << UInt32(2)"
);
// Shr
assert_eq!(
- format!("{:?}", lit(1u32) >> lit(2u32)),
+ format!("{}", lit(1u32) >> lit(2u32)),
"UInt32(1) >> UInt32(2)"
);
// Neg
- assert_eq!(format!("{:?}", -lit(1u32)), "(- UInt32(1))");
+ assert_eq!(format!("{}", -lit(1u32)), "(- UInt32(1))");
// Not
- assert_eq!(format!("{:?}", !lit(1u32)), "NOT UInt32(1)");
+ assert_eq!(format!("{}", !lit(1u32)), "NOT UInt32(1)");
}
}
diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs
index 05ebab3d99..2b6fc5793a 100644
--- a/datafusion/expr/src/utils.rs
+++ b/datafusion/expr/src/utils.rs
@@ -1291,6 +1291,7 @@ pub fn find_valid_equijoin_key_pair(
#[cfg(test)]
mod tests {
use super::*;
+ use crate::expr_vec_fmt;
use crate::{
col, cube, expr, grouping_set, rollup, AggregateFunction, WindowFrame,
WindowFunction,
@@ -1496,22 +1497,22 @@ mod tests {
// 1. col
let sets = enumerate_grouping_sets(vec![simple_col.clone()])?;
- let result = format!("{sets:?}");
+ let result = format!("[{}]", expr_vec_fmt!(sets));
assert_eq!("[simple_col]", &result);
// 2. cube
let sets = enumerate_grouping_sets(vec![cube.clone()])?;
- let result = format!("{sets:?}");
+ let result = format!("[{}]", expr_vec_fmt!(sets));
assert_eq!("[CUBE (col1, col2, col3)]", &result);
// 3. rollup
let sets = enumerate_grouping_sets(vec![rollup.clone()])?;
- let result = format!("{sets:?}");
+ let result = format!("[{}]", expr_vec_fmt!(sets));
assert_eq!("[ROLLUP (col1, col2, col3)]", &result);
// 4. col + cube
let sets = enumerate_grouping_sets(vec![simple_col.clone(), cube.clone()])?;
- let result = format!("{sets:?}");
+ let result = format!("[{}]", expr_vec_fmt!(sets));
assert_eq!(
"[GROUPING SETS (\
(simple_col), \
@@ -1527,7 +1528,7 @@ mod tests {
// 5. col + rollup
let sets = enumerate_grouping_sets(vec![simple_col.clone(), rollup.clone()])?;
- let result = format!("{sets:?}");
+ let result = format!("[{}]", expr_vec_fmt!(sets));
assert_eq!(
"[GROUPING SETS (\
(simple_col), \
@@ -1540,7 +1541,7 @@ mod tests {
// 6. col + grouping_set
let sets =
enumerate_grouping_sets(vec![simple_col.clone(), grouping_set.clone()])?;
- let result = format!("{sets:?}");
+ let result = format!("[{}]", expr_vec_fmt!(sets));
assert_eq!(
"[GROUPING SETS (\
(simple_col, col1, col2, col3))]",
@@ -1553,7 +1554,7 @@ mod tests {
grouping_set,
rollup.clone(),
])?;
- let result = format!("{sets:?}");
+ let result = format!("[{}]", expr_vec_fmt!(sets));
assert_eq!(
"[GROUPING SETS (\
(simple_col, col1, col2, col3), \
@@ -1565,7 +1566,7 @@ mod tests {
// 8. col + cube + rollup
let sets = enumerate_grouping_sets(vec![simple_col, cube, rollup])?;
- let result = format!("{sets:?}");
+ let result = format!("[{}]", expr_vec_fmt!(sets));
assert_eq!(
"[GROUPING SETS (\
(simple_col), \
diff --git a/datafusion/optimizer/src/analyzer/subquery.rs b/datafusion/optimizer/src/analyzer/subquery.rs
index e946cc22b7..c36a74c328 100644
--- a/datafusion/optimizer/src/analyzer/subquery.rs
+++ b/datafusion/optimizer/src/analyzer/subquery.rs
@@ -167,7 +167,7 @@ fn check_inner_plan(
.collect::<Vec<_>>();
if is_aggregate && is_scalar && !maybe_unsupport.is_empty() {
return Err(DataFusionError::Plan(format!(
- "Correlated column is not allowed in predicate: {predicate:?}"
+ "Correlated column is not allowed in predicate: {predicate}"
)));
}
check_inner_plan(input, is_scalar, is_aggregate, can_contain_outer_ref)
diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs
index 31f3c119be..8f551e5d80 100644
--- a/datafusion/optimizer/src/analyzer/type_coercion.rs
+++ b/datafusion/optimizer/src/analyzer/type_coercion.rs
@@ -1040,7 +1040,7 @@ mod test {
let empty = empty_with_type(DataType::Int64);
let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?);
let expected =
- "Projection: a IN ([CAST(Int32(1) AS Int64), CAST(Int8(4) AS Int64), Int64(8)]) AS a IN (Map { iter: Iter([Int32(1), Int8(4), Int64(8)]) })\
+ "Projection: a IN ([CAST(Int32(1) AS Int64), CAST(Int8(4) AS Int64), Int64(8)]) AS a IN (Map { iter: Iter([Literal(Int32(1)), Literal(Int8(4)), Literal(Int64(8))]) })\
\n EmptyRelation";
assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), &plan, expected)?;
@@ -1059,7 +1059,7 @@ mod test {
}));
let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?);
let expected =
- "Projection: CAST(a AS Decimal128(24, 4)) IN ([CAST(Int32(1) AS Decimal128(24, 4)), CAST(Int8(4) AS Decimal128(24, 4)), CAST(Int64(8) AS Decimal128(24, 4))]) AS a IN (Map { iter: Iter([Int32(1), Int8(4), Int64(8)]) })\
+ "Projection: CAST(a AS Decimal128(24, 4)) IN ([CAST(Int32(1) AS Decimal128(24, 4)), CAST(Int8(4) AS Decimal128(24, 4)), CAST(Int64(8) AS Decimal128(24, 4))]) AS a IN (Map { iter: Iter([Literal(Int32(1)), Literal(Int8(4)), Literal(Int64(8))]) })\
\n EmptyRelation";
assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), &plan, expected)
}
diff --git a/datafusion/physical-expr/src/scalar_function.rs b/datafusion/physical-expr/src/scalar_function.rs
index ef771a6784..25c0627aee 100644
--- a/datafusion/physical-expr/src/scalar_function.rs
+++ b/datafusion/physical-expr/src/scalar_function.rs
@@ -35,6 +35,7 @@ use crate::PhysicalExpr;
use arrow::datatypes::{DataType, Schema};
use arrow::record_batch::RecordBatch;
use datafusion_common::Result;
+use datafusion_expr::expr_vec_fmt;
use datafusion_expr::BuiltinScalarFunction;
use datafusion_expr::ColumnarValue;
use datafusion_expr::ScalarFunctionImplementation;
@@ -102,16 +103,7 @@ impl ScalarFunctionExpr {
impl fmt::Display for ScalarFunctionExpr {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
- write!(
- f,
- "{}({})",
- self.name,
- self.args
- .iter()
- .map(|e| format!("{e}"))
- .collect::<Vec<String>>()
- .join(", ")
- )
+ write!(f, "{}({})", self.name, expr_vec_fmt!(self.args))
}
}
diff --git a/datafusion/sql/src/utils.rs b/datafusion/sql/src/utils.rs
index 0929aec6e5..200963d321 100644
--- a/datafusion/sql/src/utils.rs
+++ b/datafusion/sql/src/utils.rs
@@ -27,6 +27,7 @@ use datafusion_expr::expr::{
WindowFunction,
};
use datafusion_expr::expr::{Cast, Sort};
+use datafusion_expr::expr_vec_fmt;
use datafusion_expr::utils::{expr_as_column_expr, find_column_exprs};
use datafusion_expr::{Expr, LogicalPlan, TryCast};
use std::collections::HashMap;
@@ -121,14 +122,10 @@ fn check_column_satisfies_expr(
) -> Result<()> {
if !columns.contains(expr) {
return Err(DataFusionError::Plan(format!(
- "{}: Expression {:?} could not be resolved from available columns: {}",
+ "{}: Expression {} could not be resolved from available columns: {}",
message_prefix,
expr,
- columns
- .iter()
- .map(|e| format!("{e}"))
- .collect::<Vec<String>>()
- .join(", ")
+ expr_vec_fmt!(columns)
)));
}
Ok(())