From ebec14ee42efd965e0d1ce25468f2239667cd5dc Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 1 Jan 2024 16:50:36 -0800 Subject: [PATCH] More --- .../enforce_distribution.rs | 228 +++++++----------- datafusion/core/tests/sql/explain_analyze.rs | 12 +- datafusion/core/tests/sql/joins.rs | 28 +-- datafusion/core/tests/sql/order.rs | 18 +- .../sqllogictest/test_files/aggregate.slt | 35 ++- .../sqllogictest/test_files/groupby.slt | 30 +-- datafusion/sqllogictest/test_files/insert.slt | 15 +- .../test_files/insert_to_external.slt | 10 +- datafusion/sqllogictest/test_files/joins.slt | 202 +++++++--------- .../test_files/repartition_scan.slt | 6 +- .../sqllogictest/test_files/tpch/q10.slt.part | 10 +- .../sqllogictest/test_files/tpch/q11.slt.part | 10 +- .../sqllogictest/test_files/tpch/q13.slt.part | 5 +- .../sqllogictest/test_files/tpch/q14.slt.part | 5 +- .../sqllogictest/test_files/tpch/q15.slt.part | 5 +- .../sqllogictest/test_files/tpch/q18.slt.part | 5 +- .../sqllogictest/test_files/tpch/q2.slt.part | 22 +- .../sqllogictest/test_files/tpch/q20.slt.part | 5 +- .../sqllogictest/test_files/tpch/q21.slt.part | 5 +- .../sqllogictest/test_files/tpch/q5.slt.part | 15 +- .../sqllogictest/test_files/tpch/q7.slt.part | 10 +- .../sqllogictest/test_files/tpch/q8.slt.part | 20 +- .../sqllogictest/test_files/tpch/q9.slt.part | 10 +- datafusion/sqllogictest/test_files/union.slt | 28 +-- datafusion/sqllogictest/test_files/window.slt | 53 ++-- 25 files changed, 318 insertions(+), 474 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 817a445950a0..4cc8cf22b124 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1202,26 +1202,14 @@ fn ensure_distribution( true }; - if enable_round_robin - // Operator benefits from partitioning (e.g. filter): - && (would_benefit && repartition_beneficial_stats) - // Unless partitioning doesn't increase the partition count, it is not beneficial: - && child.plan.output_partitioning().partition_count() < target_partitions - { - // When `repartition_file_scans` is set, attempt to increase - // parallelism at the source. - if repartition_file_scans { - if let Some(new_child) = - child.plan.repartitioned(target_partitions, config)? - { - child.plan = new_child; - } + // When `repartition_file_scans` is set, attempt to increase + // parallelism at the source. + if repartition_file_scans { + if let Some(new_child) = + child.plan.repartitioned(target_partitions, config)? + { + child.plan = new_child; } - // Increase parallelism by adding round-robin repartitioning - // on top of the operator. Note that we only do this if the - // partition count is not already equal to the desired partition - // count. - child = add_roundrobin_on_top(child, target_partitions)?; } // Satisfy the distribution requirement if it is unmet. @@ -1232,7 +1220,20 @@ fn ensure_distribution( Distribution::HashPartitioned(exprs) => { child = add_hash_on_top(child, exprs.to_vec(), target_partitions)?; } - Distribution::UnspecifiedDistribution => {} + Distribution::UnspecifiedDistribution => { + if enable_round_robin + // Operator benefits from partitioning (e.g. filter): + && (would_benefit && repartition_beneficial_stats) + // Unless partitioning doesn't increase the partition count, it is not beneficial: + && child.plan.output_partitioning().partition_count() < target_partitions + { + // Increase parallelism by adding round-robin repartitioning + // on top of the operator. Note that we only do this if the + // partition count is not already equal to the desired partition + // count. + child = add_roundrobin_on_top(child, target_partitions)?; + } + } }; // There is an ordering requirement of the operator: @@ -2076,15 +2077,12 @@ pub(crate) mod tests { JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => vec![ top_join_plan.as_str(), join_plan.as_str(), - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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 @@ -2092,15 +2090,12 @@ pub(crate) mod tests { top_join_plan.as_str(), "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", join_plan.as_str(), - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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]", ], }; @@ -2139,15 +2134,12 @@ pub(crate) mod tests { vec![ top_join_plan.as_str(), join_plan.as_str(), - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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 @@ -2156,15 +2148,12 @@ pub(crate) mod tests { top_join_plan.as_str(), "RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10", join_plan.as_str(), - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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]", ], }; @@ -2215,14 +2204,11 @@ pub(crate) mod tests { "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a1@0, c@2)]", "ProjectionExec: expr=[a@0 as a1, a@0 as a2]", "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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.clone(), true); @@ -2241,14 +2227,11 @@ pub(crate) mod tests { "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a2@1, c@2)]", "ProjectionExec: expr=[a@0 as a1, a@0 as a2]", "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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]", ]; @@ -2296,14 +2279,11 @@ pub(crate) mod tests { "ProjectionExec: expr=[c1@0 as a]", "ProjectionExec: expr=[c@2 as c1]", "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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]", ]; @@ -2498,19 +2478,15 @@ pub(crate) mod tests { "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(B@2, b1@6), (C@3, c@2), (AA@1, a1@5)]", "ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C]", "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)]", - "RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "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([b1@1, c1@2, a1@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)]", - "RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "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([b1@1, c1@2, a1@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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]", ]; @@ -2626,19 +2602,15 @@ pub(crate) mod tests { top_join_plan.as_str(), "ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C]", "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1), (c@2, c1@2)]", - "RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "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([a1@0, b1@1, c1@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)]", - "RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "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([c1@2, b1@1, a1@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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]", ]; @@ -2751,19 +2723,15 @@ pub(crate) mod tests { top_join_plan.as_str(), "ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C]", "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1)]", - "RepartitionExec: partitioning=Hash([a@0, b@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "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([a1@0, b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)]", - "RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "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([c1@2, b1@1, a1@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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]", ]; @@ -2829,17 +2797,14 @@ pub(crate) mod tests { top_join_plan.as_str(), join_plan.as_str(), "SortExec: expr=[a@0 ASC]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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 7 RepartitionExecs (4 hash, 3 round-robin), 4 SortExecs @@ -2858,17 +2823,14 @@ pub(crate) mod tests { "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", join_plan.as_str(), "SortExec: expr=[a@0 ASC]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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]", ], }; @@ -2880,17 +2842,14 @@ pub(crate) mod tests { vec![ top_join_plan.as_str(), join_plan.as_str(), - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1", "SortExec: expr=[a@0 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=1", "SortExec: expr=[b1@1 ASC]", "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([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1", "SortExec: expr=[c@2 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], @@ -2906,22 +2865,18 @@ pub(crate) mod tests { _ => vec![ top_join_plan.as_str(), // Below 4 operators are differences introduced, when join mode is changed - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1", "SortExec: expr=[a@0 ASC]", "CoalescePartitionsExec", join_plan.as_str(), - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1", "SortExec: expr=[a@0 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=1", "SortExec: expr=[b1@1 ASC]", "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([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1", "SortExec: expr=[c@2 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], @@ -2951,17 +2906,14 @@ pub(crate) mod tests { top_join_plan.as_str(), join_plan.as_str(), "SortExec: expr=[a@0 ASC]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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 7 RepartitionExecs (4 hash, 3 round-robin) and 4 SortExecs @@ -2971,17 +2923,14 @@ pub(crate) mod tests { "RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10", join_plan.as_str(), "SortExec: expr=[a@0 ASC]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(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]", ], // this match arm cannot be reached @@ -2994,39 +2943,32 @@ pub(crate) mod tests { JoinType::Inner | JoinType::Right => vec![ top_join_plan.as_str(), join_plan.as_str(), - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1", "SortExec: expr=[a@0 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=1", "SortExec: expr=[b1@1 ASC]", "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([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1", "SortExec: expr=[c@2 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], // Should include 8 RepartitionExecs (4 of them preserves order) and 4 SortExecs JoinType::Left | JoinType::Full => vec![ top_join_plan.as_str(), - "RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@6 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=1", "SortExec: expr=[b1@6 ASC]", "CoalescePartitionsExec", join_plan.as_str(), - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1", "SortExec: expr=[a@0 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=1", "SortExec: expr=[b1@1 ASC]", "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([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1", "SortExec: expr=[c@2 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], @@ -3116,8 +3058,7 @@ pub(crate) mod tests { let expected_first_sort_enforcement = &[ "SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)]", - "RepartitionExec: partitioning=Hash([b3@1, a3@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b3@1 ASC,a3@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "RepartitionExec: partitioning=Hash([b3@1, a3@0], 10), input_partitions=1", "SortExec: expr=[b3@1 ASC,a3@0 ASC]", "CoalescePartitionsExec", "ProjectionExec: expr=[a1@0 as a3, b1@1 as b3]", @@ -3127,8 +3068,7 @@ pub(crate) mod tests { "AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b2@1, a2@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b2@1 ASC,a2@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "RepartitionExec: partitioning=Hash([b2@1, a2@0], 10), input_partitions=1", "SortExec: expr=[b2@1 ASC,a2@0 ASC]", "CoalescePartitionsExec", "ProjectionExec: expr=[a@1 as a2, b@0 as b2]", @@ -3852,14 +3792,14 @@ pub(crate) mod tests { "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", // Plan already has two partitions - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e]", + "ParquetExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e]", ]; let expected_csv = [ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", // Plan already has two partitions - "CsvExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], has_header=false", + "CsvExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true, false, 2, true, 10); diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 37f8cefc9080..3ae6ab33ba95 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -660,14 +660,12 @@ async fn test_physical_plan_display_indent_multi_children() { " CoalesceBatchesExec: target_batch_size=4096", " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c1@0, c2@0)]", " CoalesceBatchesExec: target_batch_size=4096", - " 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", + " RepartitionExec: partitioning=Hash([c1@0], 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([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", + " RepartitionExec: partitioning=Hash([c2@0], 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", ]; let normalizer = ExplainNormalizer::new(); diff --git a/datafusion/core/tests/sql/joins.rs b/datafusion/core/tests/sql/joins.rs index d1f270b540b5..e7215118c92d 100644 --- a/datafusion/core/tests/sql/joins.rs +++ b/datafusion/core/tests/sql/joins.rs @@ -124,19 +124,17 @@ async fn join_change_in_planner() -> Result<()> { [ "SymmetricHashJoinExec: mode=Partitioned, 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([a2@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - // " CsvExec: file_groups={1 group: [[tempdir/left.csv]]}, projection=[a1, a2], has_header=false", + " 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([a2@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - // " CsvExec: file_groups={1 group: [[tempdir/right.csv]]}, projection=[a1, a2], has_header=false" + " RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=1", + // " CsvExec: file_groups={1 group: [[tempdir/right.csv]]}, projection=[a1, a2], has_header=false" ] }; let mut actual: Vec<&str> = formatted.trim().lines().collect(); // Remove CSV lines - actual.remove(4); - actual.remove(7); + actual.remove(3); + actual.remove(5); assert_eq!( expected, @@ -172,19 +170,17 @@ async fn join_change_in_planner_without_sort() -> Result<()> { [ "SymmetricHashJoinExec: mode=Partitioned, 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([a2@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - // " CsvExec: file_groups={1 group: [[tempdir/left.csv]]}, projection=[a1, a2], has_header=false", + " 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([a2@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - // " CsvExec: file_groups={1 group: [[tempdir/right.csv]]}, projection=[a1, a2], has_header=false" + " RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=1", + // " CsvExec: file_groups={1 group: [[tempdir/right.csv]]}, projection=[a1, a2], has_header=false" ] }; let mut actual: Vec<&str> = formatted.trim().lines().collect(); // Remove CSV lines - actual.remove(4); - actual.remove(7); + actual.remove(3); + actual.remove(5); assert_eq!( expected, diff --git a/datafusion/core/tests/sql/order.rs b/datafusion/core/tests/sql/order.rs index 6e3f6319e119..835c466b6c3b 100644 --- a/datafusion/core/tests/sql/order.rs +++ b/datafusion/core/tests/sql/order.rs @@ -208,19 +208,17 @@ 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([Int64(0)@0, t@1], 2), input_partitions=2", - " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", - " AggregateExec: mode=Partial, gby=[0 as Int64(0), t@0 as t], aggr=[]", - " ProjectionExec: expr=[column1@0 as t]", - " ValuesExec", + " RepartitionExec: partitioning=Hash([Int64(0)@0, t@1], 2), input_partitions=1", + " AggregateExec: mode=Partial, gby=[0 as Int64(0), t@0 as t], aggr=[]", + " ProjectionExec: expr=[column1@0 as t]", + " ValuesExec", " 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([Int64(1)@0, t@1], 2), input_partitions=2", - " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", - " AggregateExec: mode=Partial, gby=[1 as Int64(1), t@0 as t], aggr=[]", - " ProjectionExec: expr=[column1@0 as t]", - " ValuesExec", + " RepartitionExec: partitioning=Hash([Int64(1)@0, t@1], 2), input_partitions=1", + " AggregateExec: mode=Partial, gby=[1 as Int64(1), t@0 as t], aggr=[]", + " ProjectionExec: expr=[column1@0 as t]", + " ValuesExec", ]; let formatted = displayable(plan.as_ref()).indent(true).to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 78575c9dffc5..338d75993071 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -2541,10 +2541,9 @@ GlobalLimitExec: skip=0, fetch=4 ----SortExec: TopK(fetch=4), expr=[MAX(traces.timestamp)@1 DESC] ------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] --------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] -----------------MemoryExec: partitions=1, partition_sizes=[1] +----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 +------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] +--------------MemoryExec: partitions=1, partition_sizes=[1] query TI @@ -2596,10 +2595,9 @@ GlobalLimitExec: skip=0, fetch=4 ----SortExec: TopK(fetch=4), expr=[MAX(traces.timestamp)@1 DESC] ------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)], lim=[4] --------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)], lim=[4] -----------------MemoryExec: partitions=1, partition_sizes=[1] +----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 +------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)], lim=[4] +--------------MemoryExec: partitions=1, partition_sizes=[1] query TT explain select trace_id, MIN(timestamp) from traces group by trace_id order by MIN(timestamp) desc limit 4; @@ -2615,10 +2613,9 @@ GlobalLimitExec: skip=0, fetch=4 ----SortExec: TopK(fetch=4), expr=[MIN(traces.timestamp)@1 DESC] ------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MIN(traces.timestamp)] --------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MIN(traces.timestamp)] -----------------MemoryExec: partitions=1, partition_sizes=[1] +----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 +------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MIN(traces.timestamp)] +--------------MemoryExec: partitions=1, partition_sizes=[1] query TT explain select trace_id, MAX(timestamp) from traces group by trace_id order by MAX(timestamp) asc limit 4; @@ -2634,10 +2631,9 @@ GlobalLimitExec: skip=0, fetch=4 ----SortExec: TopK(fetch=4), expr=[MAX(traces.timestamp)@1 ASC NULLS LAST] ------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] --------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] -----------------MemoryExec: partitions=1, partition_sizes=[1] +----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 +------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] +--------------MemoryExec: partitions=1, partition_sizes=[1] query TT explain select trace_id, MAX(timestamp) from traces group by trace_id order by trace_id asc limit 4; @@ -2653,10 +2649,9 @@ GlobalLimitExec: skip=0, fetch=4 ----SortExec: TopK(fetch=4), expr=[trace_id@0 ASC NULLS LAST] ------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] --------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] -----------------MemoryExec: partitions=1, partition_sizes=[1] +----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 +------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] +--------------MemoryExec: partitions=1, partition_sizes=[1] query TI select trace_id, MAX(timestamp) from traces group by trace_id order by MAX(timestamp) desc limit 4; diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index b09ff79e88d5..37d6b7589a84 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -2752,10 +2752,9 @@ SortPreservingMergeExec: [country@0 ASC NULLS LAST] ----ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@2 as fv2] ------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] --------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ---------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] -----------------MemoryExec: partitions=1, partition_sizes=[1] +----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=1 +------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +--------------MemoryExec: partitions=1, partition_sizes=[1] query TRR SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -2788,10 +2787,9 @@ SortPreservingMergeExec: [country@0 ASC NULLS LAST] ----ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as fv2] ------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] --------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ---------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] -----------------MemoryExec: partitions=1, partition_sizes=[1] +----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=1 +------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +--------------MemoryExec: partitions=1, partition_sizes=[1] query TRR SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -4008,10 +4006,9 @@ physical_plan ProjectionExec: expr=[SUM(DISTINCT t1.x)@1 as SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)@2 as MAX(DISTINCT t1.x)] --AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)] ----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 ---------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -----------AggregateExec: mode=Partial, gby=[y@1 as y], aggr=[SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)] -------------MemoryExec: partitions=1, partition_sizes=[1] +------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=1 +--------AggregateExec: mode=Partial, gby=[y@1 as y], aggr=[SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)] +----------MemoryExec: partitions=1, partition_sizes=[1] query TT EXPLAIN SELECT SUM(DISTINCT CAST(x AS DOUBLE)), MAX(DISTINCT CAST(x AS DOUBLE)) FROM t1 GROUP BY y; @@ -4030,11 +4027,10 @@ ProjectionExec: expr=[SUM(alias1)@1 as SUM(DISTINCT t1.x), MAX(alias1)@2 as MAX( --------AggregateExec: mode=Partial, gby=[y@0 as y], aggr=[SUM(alias1), MAX(alias1)] ----------AggregateExec: mode=FinalPartitioned, gby=[y@0 as y, alias1@1 as alias1], aggr=[] ------------CoalesceBatchesExec: target_batch_size=2 ---------------RepartitionExec: partitioning=Hash([y@0, alias1@1], 8), input_partitions=8 -----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------------AggregateExec: mode=Partial, gby=[y@1 as y, CAST(t1.x AS Float64)t1.x@0 as alias1], aggr=[] ---------------------ProjectionExec: expr=[CAST(x@0 AS Float64) as CAST(t1.x AS Float64)t1.x, y@1 as y] -----------------------MemoryExec: partitions=1, partition_sizes=[1] +--------------RepartitionExec: partitioning=Hash([y@0, alias1@1], 8), input_partitions=1 +----------------AggregateExec: mode=Partial, gby=[y@1 as y, CAST(t1.x AS Float64)t1.x@0 as alias1], aggr=[] +------------------ProjectionExec: expr=[CAST(x@0 AS Float64) as CAST(t1.x AS Float64)t1.x, y@1 as y] +--------------------MemoryExec: partitions=1, partition_sizes=[1] # create an unbounded table that contains ordered timestamp. statement ok diff --git a/datafusion/sqllogictest/test_files/insert.slt b/datafusion/sqllogictest/test_files/insert.slt index e20b3779459b..5135ed743069 100644 --- a/datafusion/sqllogictest/test_files/insert.slt +++ b/datafusion/sqllogictest/test_files/insert.slt @@ -71,9 +71,8 @@ FileSinkExec: sink=MemoryTable (partitions=1) --------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }], mode=[Sorted] ----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST] ------------CoalesceBatchesExec: target_batch_size=8192 ---------------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 +--------------RepartitionExec: partitioning=Hash([c1@0], 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 query II INSERT INTO table_without_values SELECT @@ -131,9 +130,8 @@ FileSinkExec: sink=MemoryTable (partitions=1) ------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }], mode=[Sorted] --------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST] ----------CoalesceBatchesExec: target_batch_size=8192 -------------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 +------------RepartitionExec: partitioning=Hash([c1@0], 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,9 +180,8 @@ FileSinkExec: sink=MemoryTable (partitions=8) --------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }], mode=[Sorted] ----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST] ------------CoalesceBatchesExec: target_batch_size=8192 ---------------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 +--------------RepartitionExec: partitioning=Hash([c1@0], 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 query II diff --git a/datafusion/sqllogictest/test_files/insert_to_external.slt b/datafusion/sqllogictest/test_files/insert_to_external.slt index e73778ad44e5..9cc09d8b5032 100644 --- a/datafusion/sqllogictest/test_files/insert_to_external.slt +++ b/datafusion/sqllogictest/test_files/insert_to_external.slt @@ -377,9 +377,8 @@ FileSinkExec: sink=ParquetSink(file_groups=[]) --------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }], mode=[Sorted] ----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST] ------------CoalesceBatchesExec: target_batch_size=8192 ---------------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 +--------------RepartitionExec: partitioning=Hash([c1@0], 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 query II INSERT INTO table_without_values SELECT @@ -439,9 +438,8 @@ FileSinkExec: sink=ParquetSink(file_groups=[]) ------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }], mode=[Sorted] --------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST] ----------CoalesceBatchesExec: target_batch_size=8192 -------------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 +------------RepartitionExec: partitioning=Hash([c1@0], 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/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index a7146a5a91c4..3c3709b76705 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1329,13 +1329,11 @@ AggregateExec: mode=SinglePartitioned, gby=[t1_id@0 as t1_id], aggr=[] ----CoalesceBatchesExec: target_batch_size=2 ------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)] --------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] --------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] query TT EXPLAIN @@ -1357,13 +1355,11 @@ ProjectionExec: expr=[COUNT(*)@1 as COUNT(*)] ------CoalesceBatchesExec: target_batch_size=2 --------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)] ----------CoalesceBatchesExec: target_batch_size=2 -------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 ---------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------MemoryExec: partitions=1, partition_sizes=[1] +------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=1 +--------------MemoryExec: partitions=1, partition_sizes=[1] ----------CoalesceBatchesExec: target_batch_size=2 -------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 ---------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------MemoryExec: partitions=1, partition_sizes=[1] +------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=1 +--------------MemoryExec: partitions=1, partition_sizes=[1] query TT EXPLAIN @@ -1390,13 +1386,11 @@ ProjectionExec: expr=[COUNT(alias1)@0 as COUNT(DISTINCT join_t1.t1_id)] --------------CoalesceBatchesExec: target_batch_size=2 ----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)] ------------------CoalesceBatchesExec: target_batch_size=2 ---------------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------------------MemoryExec: partitions=1, partition_sizes=[1] +--------------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=1 +----------------------MemoryExec: partitions=1, partition_sizes=[1] ------------------CoalesceBatchesExec: target_batch_size=2 ---------------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------------------MemoryExec: partitions=1, partition_sizes=[1] +--------------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=1 +----------------------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.explain.logical_plan_only = true; @@ -1597,9 +1591,8 @@ ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] --CoalesceBatchesExec: target_batch_size=2 ----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)] ------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] ------CoalesceBatchesExec: target_batch_size=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)] @@ -1659,9 +1652,8 @@ ProjectionExec: expr=[t1_id@2 as t1_id, t2_id@0 as t2_id, t1_name@3 as t1_name] ------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 --------------MemoryExec: partitions=1, partition_sizes=[1] ------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +--------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] # Select wildcard with expr key inner join @@ -1707,9 +1699,8 @@ ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int --CoalesceBatchesExec: target_batch_size=2 ----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)] ------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +--------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] ------CoalesceBatchesExec: target_batch_size=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)] @@ -2553,13 +2544,11 @@ physical_plan CoalesceBatchesExec: target_batch_size=2 --HashJoinExec: mode=Partitioned, join_type=Inner, on=[(millis@2, millis@2)] ----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([millis@2], 2), input_partitions=2 ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +------RepartitionExec: partitioning=Hash([millis@2], 2), input_partitions=1 +--------MemoryExec: partitions=1, partition_sizes=[1] ----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([millis@2], 2), input_partitions=2 ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +------RepartitionExec: partitioning=Hash([millis@2], 2), input_partitions=1 +--------MemoryExec: partitions=1, partition_sizes=[1] # left_join_using_2 query II @@ -2726,14 +2715,12 @@ physical_plan SortMergeJoin: join_type=Inner, on=[(c1@0, c1@0)] --SortExec: expr=[c1@0 ASC] ----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=1 +--------MemoryExec: partitions=1, partition_sizes=[1] --SortExec: expr=[c1@0 ASC] ----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=1 +--------MemoryExec: partitions=1, partition_sizes=[1] # sort_merge_join_on_date32 inner sort merge join on data type (Date32) query DDR?DDR? rowsort @@ -2763,9 +2750,8 @@ ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, c1@5 as c1 --------------MemoryExec: partitions=1, partition_sizes=[1] ----SortExec: expr=[c3@2 ASC] ------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([c3@2], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +--------RepartitionExec: partitioning=Hash([c3@2], 2), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] # sort_merge_join_on_decimal right join on data type (Decimal) query DDR?DDR? rowsort @@ -2821,13 +2807,11 @@ SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] ----CoalesceBatchesExec: target_batch_size=2 ------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] --------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] --------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] query IT rowsort SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 WHERE t1_id IN (SELECT t2_id FROM left_semi_anti_join_table_t2 t2) ORDER BY t1_id @@ -2862,13 +2846,11 @@ SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] ----CoalesceBatchesExec: target_batch_size=2 ------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] --------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] --------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] query IT SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 LEFT SEMI JOIN left_semi_anti_join_table_t2 t2 ON (t1_id = t2_id) ORDER BY t1_id @@ -3018,13 +3000,11 @@ SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] ----CoalesceBatchesExec: target_batch_size=2 ------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 --------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] --------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] query ITI rowsort SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t1 t1 WHERE EXISTS (SELECT * FROM right_semi_anti_join_table_t2 t2 where t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id @@ -3040,13 +3020,11 @@ SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] ----CoalesceBatchesExec: target_batch_size=2 ------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 --------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] --------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] query ITI rowsort SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t2 t2 RIGHT SEMI JOIN right_semi_anti_join_table_t1 t1 on (t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id @@ -3192,18 +3170,14 @@ Sort: l_table.rn1 ASC NULLS LAST physical_plan SortPreservingMergeExec: [rn1@5 ASC NULLS LAST] --SortMergeJoin: join_type=Inner, on=[(a@1, a@1)] -----SortExec: expr=[rn1@5 ASC NULLS LAST] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] ---------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }], mode=[Sorted] -----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true -----SortExec: expr=[a@1 ASC] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +----CoalesceBatchesExec: target_batch_size=2 +------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=1 +--------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] +----------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }], mode=[Sorted] ------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +----CoalesceBatchesExec: target_batch_size=2 +------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=1 +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true # sort merge join should propagate ordering equivalence of the right side # for right join. Hence final requirement rn1 ASC is already satisfied at @@ -3228,18 +3202,14 @@ Sort: r_table.rn1 ASC NULLS LAST physical_plan SortPreservingMergeExec: [rn1@10 ASC NULLS LAST] --SortMergeJoin: join_type=Right, on=[(a@1, a@1)] -----SortExec: expr=[a@1 ASC] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +----CoalesceBatchesExec: target_batch_size=2 +------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=1 +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +----CoalesceBatchesExec: target_batch_size=2 +------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=1 +--------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] +----------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }], mode=[Sorted] ------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true -----SortExec: expr=[rn1@5 ASC NULLS LAST] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] ---------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }], mode=[Sorted] -----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true # SortMergeJoin should add ordering equivalences of # right table as lexicographical append to the global ordering @@ -3267,22 +3237,17 @@ Sort: l_table.a ASC NULLS FIRST, l_table.b ASC NULLS LAST, l_table.c ASC NULLS L ----------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan SortPreservingMergeExec: [a@1 ASC,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,rn1@11 ASC NULLS LAST] ---SortExec: expr=[a@1 ASC,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,rn1@11 ASC NULLS LAST] -----SortMergeJoin: join_type=Inner, on=[(a@1, a@1)] -------SortExec: expr=[a@1 ASC] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] -----------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }], mode=[Sorted] -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true -------SortExec: expr=[a@1 ASC] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] -----------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }], mode=[Sorted] -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +--SortMergeJoin: join_type=Inner, on=[(a@1, a@1)] +----CoalesceBatchesExec: target_batch_size=2 +------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=1 +--------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] +----------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }], mode=[Sorted] +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +----CoalesceBatchesExec: target_batch_size=2 +------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=1 +--------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] +----------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }], mode=[Sorted] +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true statement ok set datafusion.optimizer.prefer_hash_join = true; @@ -3449,22 +3414,19 @@ Sort: l.a ASC NULLS FIRST ----------TableScan: annotated_data projection=[a, b] physical_plan SortPreservingMergeExec: [a@0 ASC] ---SortExec: expr=[a@0 ASC] -----ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] -------AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 2), input_partitions=2 -------------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)] ---------------CoalesceBatchesExec: target_batch_size=2 -----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)] -------------------CoalesceBatchesExec: target_batch_size=2 ---------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 -----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true -------------------CoalesceBatchesExec: target_batch_size=2 ---------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 -----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], has_header=true +--ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] +----AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)], ordering_mode=PartiallySorted([0]) +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 2), input_partitions=2, preserve_order=true, sort_exprs=a@0 ASC +----------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)], ordering_mode=PartiallySorted([0]) +------------CoalesceBatchesExec: target_batch_size=2 +--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)] +----------------CoalesceBatchesExec: target_batch_size=2 +------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=1 +--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +----------------CoalesceBatchesExec: target_batch_size=2 +------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=1 +--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], has_header=true query TT EXPLAIN SELECT * diff --git a/datafusion/sqllogictest/test_files/repartition_scan.slt b/datafusion/sqllogictest/test_files/repartition_scan.slt index 02eccd7c5d06..d1ead0b03202 100644 --- a/datafusion/sqllogictest/test_files/repartition_scan.slt +++ b/datafusion/sqllogictest/test_files/repartition_scan.slt @@ -149,11 +149,11 @@ LOCATION 'test_files/scratch/repartition_scan/csv_table/'; query I select * from csv_table; ---- -1 +5 2 3 4 -5 +1 ## Expect to see the scan read the file as "4" groups with even sizes (offsets) query TT @@ -192,11 +192,11 @@ LOCATION 'test_files/scratch/repartition_scan/json_table/'; query I select * from "json_table"; ---- +5 1 2 3 4 -5 ## Expect to see the scan read the file as "4" groups with even sizes (offsets) query TT diff --git a/datafusion/sqllogictest/test_files/tpch/q10.slt.part b/datafusion/sqllogictest/test_files/tpch/q10.slt.part index eb0b66f024de..04e3bf6c8b72 100644 --- a/datafusion/sqllogictest/test_files/tpch/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q10.slt.part @@ -92,9 +92,8 @@ GlobalLimitExec: skip=0, fetch=10 --------------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)] ------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------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/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment], has_header=false +--------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 +----------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/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([o_custkey@1], 4), input_partitions=4 ----------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] @@ -108,9 +107,8 @@ GlobalLimitExec: skip=0, fetch=10 ----------------------------------------FilterExec: l_returnflag@3 = R ------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], has_header=false ----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------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/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/q11.slt.part index 4efa29e2c0ac..d862a464c596 100644 --- a/datafusion/sqllogictest/test_files/tpch/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q11.slt.part @@ -95,9 +95,8 @@ GlobalLimitExec: skip=0, fetch=10 ------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 --------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost], has_header=false ----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------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/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +--------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false ------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 ----------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] @@ -121,9 +120,8 @@ GlobalLimitExec: skip=0, fetch=10 ------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 --------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], has_header=false ----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------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/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +--------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false ------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 ----------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] diff --git a/datafusion/sqllogictest/test_files/tpch/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/q13.slt.part index 5cf6ace8b27b..3d69cc689922 100644 --- a/datafusion/sqllogictest/test_files/tpch/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q13.slt.part @@ -68,9 +68,8 @@ GlobalLimitExec: skip=0, fetch=10 ----------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)] --------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------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/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], has_header=false +----------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 +------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], has_header=false --------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 ------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] diff --git a/datafusion/sqllogictest/test_files/tpch/q14.slt.part b/datafusion/sqllogictest/test_files/tpch/q14.slt.part index b584972c25bc..3af5fee8c12d 100644 --- a/datafusion/sqllogictest/test_files/tpch/q14.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q14.slt.part @@ -55,9 +55,8 @@ ProjectionExec: expr=[100 * CAST(SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") ----------------------FilterExec: l_shipdate@3 >= 9374 AND l_shipdate@3 < 9404 ------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], has_header=false --------------CoalesceBatchesExec: target_batch_size=8192 -----------------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/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false +----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=1 +------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q15.slt.part b/datafusion/sqllogictest/test_files/tpch/q15.slt.part index a872e96acf04..9a6e188a30bf 100644 --- a/datafusion/sqllogictest/test_files/tpch/q15.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q15.slt.part @@ -82,9 +82,8 @@ SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST] ----------------CoalesceBatchesExec: target_batch_size=8192 ------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, supplier_no@0)] --------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------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/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_phone], has_header=false +----------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/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 diff --git a/datafusion/sqllogictest/test_files/tpch/q18.slt.part b/datafusion/sqllogictest/test_files/tpch/q18.slt.part index 53191a5d44e1..287470edcbdc 100644 --- a/datafusion/sqllogictest/test_files/tpch/q18.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q18.slt.part @@ -84,9 +84,8 @@ SortPreservingMergeExec: [o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST] ----------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)] --------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------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/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name], has_header=false +----------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 +------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name], has_header=false --------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 ------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/q2.slt.part index ed439348d22d..e2541e11b6b2 100644 --- a/datafusion/sqllogictest/test_files/tpch/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q2.slt.part @@ -137,13 +137,11 @@ GlobalLimitExec: skip=0, fetch=10 ------------------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 --------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false ------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------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/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], has_header=false +--------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +----------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/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([n_nationkey@0], 4), input_partitions=4 -------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], has_header=false +----------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], has_header=false ----------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 --------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] @@ -175,13 +173,11 @@ GlobalLimitExec: skip=0, fetch=10 ------------------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 --------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false ----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------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/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +--------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false ------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------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/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false +--------------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +----------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false --------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 ------------------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] @@ -238,7 +234,7 @@ order by p_partkey limit 10; ---- -9828.21 Supplier#000000647 UNITED KINGDOM 13120 Manufacturer#5 x5U7MBZmwfG9 33-258-202-4782 s the slyly even ideas poach fluffily +9828.21 Supplier#000000647 UNITED KINGDOM 13120 Manufacturer#5 x5U7MBZmwfG9 33-258-202-4782 s the slyly even ideas poach fluffily 9508.37 Supplier#000000070 FRANCE 3563 Manufacturer#1 INWNH2w,OOWgNDq0BRCcBwOMQc6PdFDc4 16-821-608-1166 ests sleep quickly express ideas. ironic ideas haggle about the final T 9508.37 Supplier#000000070 FRANCE 17268 Manufacturer#4 INWNH2w,OOWgNDq0BRCcBwOMQc6PdFDc4 16-821-608-1166 ests sleep quickly express ideas. ironic ideas haggle about the final T 9453.01 Supplier#000000802 ROMANIA 10021 Manufacturer#5 ,6HYXb4uaHITmtMBj4Ak57Pd 29-342-882-6463 gular frets. permanently special multipliers believe blithely alongs diff --git a/datafusion/sqllogictest/test_files/tpch/q20.slt.part b/datafusion/sqllogictest/test_files/tpch/q20.slt.part index e014c6cafd98..680132d16663 100644 --- a/datafusion/sqllogictest/test_files/tpch/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q20.slt.part @@ -92,9 +92,8 @@ SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] ----------------CoalesceBatchesExec: target_batch_size=8192 ------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)] --------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------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/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey], has_header=false +----------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=1 +------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/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([n_nationkey@0], 4), input_partitions=4 ------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] diff --git a/datafusion/sqllogictest/test_files/tpch/q21.slt.part b/datafusion/sqllogictest/test_files/tpch/q21.slt.part index 147afc603c2c..4eb2a2eea190 100644 --- a/datafusion/sqllogictest/test_files/tpch/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q21.slt.part @@ -118,9 +118,8 @@ SortPreservingMergeExec: [numwait@1 DESC,s_name@0 ASC NULLS LAST] --------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)] ------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------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/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_nationkey], has_header=false +--------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +----------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_nationkey], has_header=false ------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 ----------------------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] diff --git a/datafusion/sqllogictest/test_files/tpch/q5.slt.part b/datafusion/sqllogictest/test_files/tpch/q5.slt.part index af3a33497026..afc4451eb22e 100644 --- a/datafusion/sqllogictest/test_files/tpch/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q5.slt.part @@ -98,9 +98,8 @@ SortPreservingMergeExec: [revenue@1 DESC] --------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)] ------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------------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/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false +--------------------------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 +----------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false ------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 ----------------------------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] @@ -111,13 +110,11 @@ SortPreservingMergeExec: [revenue@1 DESC] ----------------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 ------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount], has_header=false ----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------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/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=1 +--------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false ------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------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/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], has_header=false +--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], has_header=false --------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 ------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] diff --git a/datafusion/sqllogictest/test_files/tpch/q7.slt.part b/datafusion/sqllogictest/test_files/tpch/q7.slt.part index 5186c46a896f..0351457a5ff8 100644 --- a/datafusion/sqllogictest/test_files/tpch/q7.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q7.slt.part @@ -115,9 +115,8 @@ SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS L --------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)] ------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------------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/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +--------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +----------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false ------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 ----------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 @@ -127,9 +126,8 @@ SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS L ----------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 ------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey], has_header=false ----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------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/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false +------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 +--------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false ------------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 ----------------------------------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/q8.slt.part index 760b40ad1ae8..f9642926f389 100644 --- a/datafusion/sqllogictest/test_files/tpch/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q8.slt.part @@ -141,26 +141,22 @@ SortPreservingMergeExec: [o_year@0 ASC NULLS LAST] ----------------------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 ------------------------------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount], has_header=false ----------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------------------------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/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +------------------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +--------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false ------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------------------------------------------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 --------------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false --------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------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/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false +----------------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 +------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false ----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------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/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false +------------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +--------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false ------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------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/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false --------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 ------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] diff --git a/datafusion/sqllogictest/test_files/tpch/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/q9.slt.part index 5db97f79bdb1..b1623a83dd62 100644 --- a/datafusion/sqllogictest/test_files/tpch/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q9.slt.part @@ -118,9 +118,8 @@ GlobalLimitExec: skip=0, fetch=10 ----------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 ------------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], has_header=false ----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------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/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +--------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false ------------------------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 ----------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false @@ -128,9 +127,8 @@ GlobalLimitExec: skip=0, fetch=10 ----------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 ------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate], has_header=false ----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------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/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index b4e338875e24..a8105dc4324c 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -205,14 +205,13 @@ Aggregate: groupBy=[[t1.name]], aggr=[[]] physical_plan AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] --CoalesceBatchesExec: target_batch_size=8192 -----RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 ---------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] -----------UnionExec +----RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=3 +------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] +--------UnionExec +----------MemoryExec: partitions=1, partition_sizes=[1] +----------MemoryExec: partitions=1, partition_sizes=[1] +----------ProjectionExec: expr=[name@0 || _new as name] ------------MemoryExec: partitions=1, partition_sizes=[1] -------------MemoryExec: partitions=1, partition_sizes=[1] -------------ProjectionExec: expr=[name@0 || _new as name] ---------------MemoryExec: partitions=1, partition_sizes=[1] # nested_union_all query T rowsort @@ -303,9 +302,8 @@ UnionExec ----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ------------------------MemoryExec: partitions=1, partition_sizes=[1] --------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +----------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] query IT rowsort @@ -360,9 +358,8 @@ InterleaveExec --------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ----------------MemoryExec: partitions=1, partition_sizes=[1] ------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +--------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] --CoalesceBatchesExec: target_batch_size=2 ----HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(name@0, name@0)] ------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] @@ -372,9 +369,8 @@ InterleaveExec --------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ----------------MemoryExec: partitions=1, partition_sizes=[1] ------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +--------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] # union_upcast_types query TT diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index aa083290b4f4..e318644b71eb 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1287,9 +1287,8 @@ ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_10 ------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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([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 +------------------RepartitionExec: partitioning=Hash([c1@0, c2@1], 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 # test_window_agg_sort_reversed_plan @@ -1789,16 +1788,15 @@ Limit: skip=0, fetch=5 physical_plan GlobalLimitExec: skip=0, fetch=5 --SortPreservingMergeExec: [c3@0 ASC NULLS LAST], fetch=5 -----ProjectionExec: expr=[c3@0 as c3, 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@2 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum2] -------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow }], mode=[Sorted] ---------SortExec: expr=[c3@0 ASC NULLS LAST,c9@1 DESC] +----SortExec: TopK(fetch=5), expr=[c3@0 ASC NULLS LAST] +------ProjectionExec: expr=[c3@0 as c3, 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@2 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum2] +--------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow }], mode=[Sorted] ----------CoalesceBatchesExec: target_batch_size=4096 -------------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) 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] -------------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int16(NULL)), end_bound: CurrentRow }], mode=[Sorted] ---------------------SortExec: expr=[c3@1 DESC,c9@2 DESC,c2@0 ASC NULLS LAST] -----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c9], has_header=true +------------RepartitionExec: partitioning=Hash([c3@0], 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) 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] +----------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int16(NULL)), end_bound: CurrentRow }], mode=[Sorted] +------------------SortExec: expr=[c3@1 DESC,c9@2 DESC,c2@0 ASC NULLS LAST] +--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c9], has_header=true @@ -1810,8 +1808,8 @@ SELECT c3, ORDER BY c3 LIMIT 5 ---- --117 219796664156 3023531799 -117 222089770060 5316637703 +-117 219796664156 3023531799 -111 216773132357 1243785310 -107 215529347047 1157161427 -106 214372185620 141680161 @@ -1835,9 +1833,8 @@ SortPreservingMergeExec: [c1@0 ASC NULLS LAST] ----BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }], mode=[Sorted] ------SortExec: expr=[c1@0 ASC NULLS LAST] --------CoalesceBatchesExec: target_batch_size=4096 -----------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 +----------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=1 +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true query TI SELECT c1, ROW_NUMBER() OVER (PARTITION BY c1) as rn1 FROM aggregate_test_100 ORDER BY c1 ASC @@ -1964,9 +1961,8 @@ SortPreservingMergeExec: [c1@0 ASC NULLS LAST,rn1@1 ASC NULLS LAST] ----BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }], mode=[Sorted] ------SortExec: expr=[c1@0 ASC NULLS LAST] --------CoalesceBatchesExec: target_batch_size=4096 -----------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 +----------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=1 +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true statement ok set datafusion.optimizer.repartition_sorts = true; @@ -1993,9 +1989,8 @@ SortExec: expr=[c1@0 ASC NULLS LAST] ----------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 3 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 3 FOLLOWING", 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([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 +----------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=1 +------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true # test_window_agg_with_global_limit statement ok @@ -3255,9 +3250,8 @@ ProjectionExec: expr=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_da --------------------RepartitionExec: partitioning=Hash([a@0, d@3], 2), input_partitions=2, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,c@2 ASC NULLS LAST ----------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] ------------------------CoalesceBatchesExec: target_batch_size=4096 ---------------------------RepartitionExec: partitioning=Hash([a@0, b@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,c@2 ASC NULLS LAST -----------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------------------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] +--------------------------RepartitionExec: partitioning=Hash([a@0, b@1], 2), input_partitions=1 +----------------------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] # reset the partition number 1 again statement ok @@ -3571,9 +3565,8 @@ SortPreservingMergeExec: [c@3 ASC NULLS LAST] --ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW@5 as avg_d] ----BoundedWindowAggExec: wdw=[AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW: Ok(Field { name: "AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: CurrentRow }], mode=[Linear] ------CoalesceBatchesExec: target_batch_size=4096 ---------RepartitionExec: partitioning=Hash([d@4], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST] +--------RepartitionExec: partitioning=Hash([d@4], 2), input_partitions=1 +----------StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST] # CTAS with NTILE function statement ok @@ -3794,7 +3787,7 @@ select a, 1 1 2 1 -# support scalar value in ORDER BY +# support scalar value in ORDER BY query I select rank() over (order by 1) rnk from (select 1 a union all select 2 a) x ---- @@ -3832,4 +3825,4 @@ select row_number() over (partition by 1 order by 1) rn, from (select 1 a union all select 2 a) x; ---- 1 1 1 1 1 1 -2 1 1 2 2 1 \ No newline at end of file +2 1 1 2 2 1