diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 05f5a204c096..aab51efa1e34 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -2864,10 +2864,9 @@ async fn test_count_wildcard_on_sort() -> Result<()> { | | ProjectionExec: expr=[b@0 as b, count(Int64(1))@1 as count(*), count(Int64(1))@1 as count(Int64(1))] | | | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(Int64(1))] | | | CoalesceBatchesExec: target_batch_size=8192 | - | | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 | - | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | - | | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(Int64(1))] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=1 | + | | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(Int64(1))] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | | | | +---------------+------------------------------------------------------------------------------------------------------------+ "### @@ -2876,22 +2875,21 @@ async fn test_count_wildcard_on_sort() -> Result<()> { assert_snapshot!( pretty_format_batches(&df_results).unwrap(), @r###" - +---------------+--------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+--------------------------------------------------------------------------------+ - | logical_plan | Sort: count(*) ASC NULLS LAST | - | | Aggregate: groupBy=[[t1.b]], aggr=[[count(Int64(1)) AS count(*)]] | - | | TableScan: t1 projection=[b] | - | physical_plan | SortPreservingMergeExec: [count(*)@1 ASC NULLS LAST] | - | | SortExec: expr=[count(*)@1 ASC NULLS LAST], preserve_partitioning=[true] | - | | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(*)] | - | | CoalesceBatchesExec: target_batch_size=8192 | - | | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 | - | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | - | | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(*)] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+--------------------------------------------------------------------------------+ + +---------------+----------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+----------------------------------------------------------------------------+ + | logical_plan | Sort: count(*) ASC NULLS LAST | + | | Aggregate: groupBy=[[t1.b]], aggr=[[count(Int64(1)) AS count(*)]] | + | | TableScan: t1 projection=[b] | + | physical_plan | SortPreservingMergeExec: [count(*)@1 ASC NULLS LAST] | + | | SortExec: expr=[count(*)@1 ASC NULLS LAST], preserve_partitioning=[true] | + | | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(*)] | + | | CoalesceBatchesExec: target_batch_size=8192 | + | | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=1 | + | | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(*)] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+----------------------------------------------------------------------------+ "### ); Ok(()) @@ -3351,10 +3349,9 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { | | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] | | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] | | | CoalesceBatchesExec: target_batch_size=8192 | - | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 | - | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | - | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 | + | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | | | | +---------------+---------------------------------------------------------------------------------------------------------------------------+ " @@ -3408,10 +3405,9 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { | | ProjectionExec: expr=[count(*)@1 as count(*), a@0 as a, true as __always_true] | | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(*)] | | | CoalesceBatchesExec: target_batch_size=8192 | - | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 | - | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | - | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(*)] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 | + | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(*)] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | | | | +---------------+---------------------------------------------------------------------------------------------------------------------------+ " diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 5b7d9ac8fbe9..354983e08e2a 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -618,16 +618,13 @@ fn multi_hash_joins() -> Result<()> { assert_plan!(plan_distrib, @r" HashJoinExec: mode=Partitioned, join_type=..., on=[(a@0, c@2)] HashJoinExec: mode=Partitioned, join_type=..., on=[(a@0, b1@1)] - RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet "); }, // Should include 4 RepartitionExecs @@ -636,16 +633,13 @@ fn multi_hash_joins() -> Result<()> { HashJoinExec: mode=Partitioned, join_type=..., on=[(a@0, c@2)] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 HashJoinExec: mode=Partitioned, join_type=..., on=[(a@0, b1@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 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + 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] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet "); }, }; @@ -690,16 +684,13 @@ fn multi_hash_joins() -> Result<()> { assert_plan!(plan_distrib, @r" HashJoinExec: mode=Partitioned, join_type=..., on=[(b1@1, c@2)] HashJoinExec: mode=Partitioned, join_type=..., on=[(a@0, b1@1)] - RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet "); } @@ -710,16 +701,13 @@ fn multi_hash_joins() -> Result<()> { HashJoinExec: mode=Partitioned, join_type=..., on=[(b1@6, c@2)] RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10 HashJoinExec: mode=Partitioned, join_type=..., on=[(a@0, b1@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 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + 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] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet "); }, @@ -780,15 +768,12 @@ fn multi_joins_after_alias() -> Result<()> { HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a1@0, c@2)] ProjectionExec: expr=[a@0 as a1, a@0 as a2] HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)] - RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); let plan_sort = test_config.to_plan(top_join, &SORT_DISTRIB_DISTRIB); @@ -811,15 +796,12 @@ fn multi_joins_after_alias() -> Result<()> { HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a2@1, c@2)] ProjectionExec: expr=[a@0 as a1, a@0 as a2] HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)] - RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); let plan_sort = test_config.to_plan(top_join, &SORT_DISTRIB_DISTRIB); @@ -869,15 +851,12 @@ fn multi_joins_after_multi_alias() -> Result<()> { ProjectionExec: expr=[c1@0 as a] ProjectionExec: expr=[c@2 as c1] HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)] - RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); let plan_sort = test_config.to_plan(top_join, &SORT_DISTRIB_DISTRIB); @@ -1098,21 +1077,17 @@ fn multi_hash_join_key_ordering() -> Result<()> { HashJoinExec: mode=Partitioned, join_type=Inner, on=[(B@2, b1@6), (C@3, c@2), (AA@1, a1@5)] ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C] HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)] - RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)] - RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + 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] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)] + RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); let plan_sort = test_config.to_plan(filter_top_join, &SORT_DISTRIB_DISTRIB); @@ -1239,21 +1214,17 @@ fn reorder_join_keys_to_left_input() -> Result<()> { HashJoinExec: mode=Partitioned, join_type=..., on=[(AA@1, a1@5), (B@2, b1@6), (C@3, c@2)] ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C] HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1), (c@2, c1@2)] - RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([a1@0, b1@1, c1@2], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)] - RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + 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] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)] + RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet ");} } @@ -1371,21 +1342,17 @@ fn reorder_join_keys_to_right_input() -> Result<()> { HashJoinExec: mode=Partitioned, join_type=..., on=[(C@3, c@2), (B@2, b1@6), (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=[(a@0, a1@0), (b@1, b1@1)] - RepartitionExec: partitioning=Hash([a@0, b@1], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([a1@0, b1@1], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)] - RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + RepartitionExec: partitioning=Hash([a@0, b@1], 10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + 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] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)] + RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet ");} } @@ -1450,18 +1417,15 @@ fn multi_smj_joins() -> Result<()> { SortMergeJoin: join_type=..., on=[(a@0, c@2)] SortMergeJoin: join_type=..., on=[(a@0, b1@1)] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] - RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet SortExec: expr=[b1@1 ASC], preserve_partitioning=[true] - RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + 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] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet SortExec: expr=[c@2 ASC], preserve_partitioning=[true] - RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet "); } // Should include 7 RepartitionExecs (4 hash, 3 round-robin), 4 SortExecs @@ -1480,18 +1444,15 @@ SortMergeJoin: join_type=..., on=[(a@0, c@2)] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 SortMergeJoin: join_type=..., on=[(a@0, b1@1)] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] - RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet SortExec: expr=[b1@1 ASC], preserve_partitioning=[true] - RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + 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] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet SortExec: expr=[c@2 ASC], preserve_partitioning=[true] - RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet "); } } @@ -1505,19 +1466,16 @@ SortMergeJoin: join_type=..., on=[(a@0, c@2)] assert_plan!(plan_sort, @r" SortMergeJoin: join_type=..., on=[(a@0, c@2)] SortMergeJoin: join_type=..., on=[(a@0, b1@1)] - RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - SortExec: expr=[b1@1 ASC], preserve_partitioning=[false] - ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1 + SortExec: expr=[a@0 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=1 + SortExec: expr=[b1@1 ASC], preserve_partitioning=[false] + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1 + SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet "); } // Should include 8 RepartitionExecs (4 hash, 8 round-robin), 4 SortExecs @@ -1533,24 +1491,20 @@ SortMergeJoin: join_type=..., on=[(a@0, c@2)] // TODO(wiedld): show different test result if enforce distribution first. assert_plan!(plan_sort, @r" SortMergeJoin: join_type=..., on=[(a@0, c@2)] - RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] - CoalescePartitionsExec - SortMergeJoin: join_type=..., on=[(a@0, b1@1)] - RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - SortExec: expr=[b1@1 ASC], preserve_partitioning=[false] - ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - SortExec: expr=[c@2 ASC], preserve_partitioning=[false] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1 + SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + CoalescePartitionsExec + SortMergeJoin: join_type=..., on=[(a@0, b1@1)] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1 + SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=1 + SortExec: expr=[b1@1 ASC], preserve_partitioning=[false] + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1 + SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet "); } } @@ -1575,18 +1529,15 @@ SortMergeJoin: join_type=..., on=[(a@0, c@2)] SortMergeJoin: join_type=..., on=[(b1@6, c@2)] SortMergeJoin: join_type=..., on=[(a@0, b1@1)] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] - RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet SortExec: expr=[b1@1 ASC], preserve_partitioning=[true] - RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + 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] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet SortExec: expr=[c@2 ASC], preserve_partitioning=[true] - RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet "); } // Should include 7 RepartitionExecs (4 hash, 3 round-robin) and 4 SortExecs @@ -1598,18 +1549,15 @@ SortMergeJoin: join_type=..., on=[(b1@6, c@2)] RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10 SortMergeJoin: join_type=..., on=[(a@0, b1@1)] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] - RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet SortExec: expr=[b1@1 ASC], preserve_partitioning=[true] - RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + 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] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet SortExec: expr=[c@2 ASC], preserve_partitioning=[true] - RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet "); } // this match arm cannot be reached @@ -1625,19 +1573,16 @@ SortMergeJoin: join_type=..., on=[(b1@6, c@2)] assert_plan!(plan_sort, @r" SortMergeJoin: join_type=..., on=[(b1@6, c@2)] SortMergeJoin: join_type=..., on=[(a@0, b1@1)] - RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - SortExec: expr=[b1@1 ASC], preserve_partitioning=[false] - ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1 + SortExec: expr=[a@0 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=1 + SortExec: expr=[b1@1 ASC], preserve_partitioning=[false] + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1 + SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet "); } // Should include 8 RepartitionExecs (4 of them preserves order) and 4 SortExecs @@ -1645,24 +1590,20 @@ SortMergeJoin: join_type=..., on=[(b1@6, c@2)] // TODO(wiedld): show different test result if enforce distribution first. assert_plan!(plan_sort, @r" SortMergeJoin: join_type=..., on=[(b1@6, c@2)] - RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@6 ASC - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - SortExec: expr=[b1@6 ASC], preserve_partitioning=[false] - CoalescePartitionsExec - SortMergeJoin: join_type=..., on=[(a@0, b1@1)] - RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - SortExec: expr=[a@0 ASC], preserve_partitioning=[false] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - SortExec: expr=[b1@1 ASC], preserve_partitioning=[false] - ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - SortExec: expr=[c@2 ASC], preserve_partitioning=[false] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=1 + SortExec: expr=[b1@6 ASC], preserve_partitioning=[false] + CoalescePartitionsExec + SortMergeJoin: join_type=..., on=[(a@0, b1@1)] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1 + SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=1 + SortExec: expr=[b1@1 ASC], preserve_partitioning=[false] + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1 + SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet "); } // this match arm cannot be reached @@ -1753,27 +1694,25 @@ SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)] let plan_sort = test_config.to_plan(join, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_sort, @r" SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)] - RepartitionExec: partitioning=Hash([b3@1, a3@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b3@1 ASC, a3@0 ASC - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - SortExec: expr=[b3@1 ASC, a3@0 ASC], preserve_partitioning=[false] - CoalescePartitionsExec - ProjectionExec: expr=[a1@0 as a3, b1@1 as b3] - ProjectionExec: expr=[a1@1 as a1, b1@0 as b1] - AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[] - RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[] - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([b2@1, a2@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b2@1 ASC, a2@0 ASC - RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[false] - CoalescePartitionsExec - ProjectionExec: expr=[a@1 as a2, b@0 as b2] - AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[] - RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[] + RepartitionExec: partitioning=Hash([b3@1, a3@0], 10), input_partitions=1 + SortExec: expr=[b3@1 ASC, a3@0 ASC], preserve_partitioning=[false] + CoalescePartitionsExec + ProjectionExec: expr=[a1@0 as a3, b1@1 as b3] + ProjectionExec: expr=[a1@1 as a1, b1@0 as b1] + AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[] + RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10 + AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b2@1, a2@0], 10), input_partitions=1 + SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[false] + CoalescePartitionsExec + ProjectionExec: expr=[a@1 as a2, b@0 as b2] + AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[] + RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10 + AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[] + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet "); Ok(()) diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 26b71b5496f2..d9ea97886ba1 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -798,14 +798,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)], projection=[c1@0] CoalesceBatchesExec: target_batch_size=4096 - RepartitionExec: partitioning=Hash([c1@0], 9000), input_partitions=9000 - RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1 - DataSourceExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true + RepartitionExec: partitioning=Hash([c1@0], 9000), input_partitions=1 + DataSourceExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, 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] - DataSourceExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true + RepartitionExec: partitioning=Hash([c2@0], 9000), input_partitions=1 + ProjectionExec: expr=[c1@0 as c2] + DataSourceExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true "### ); } diff --git a/datafusion/core/tests/sql/joins.rs b/datafusion/core/tests/sql/joins.rs index 7a5983447592..b64a1e18f3f6 100644 --- a/datafusion/core/tests/sql/joins.rs +++ b/datafusion/core/tests/sql/joins.rs @@ -73,13 +73,11 @@ async fn join_change_in_planner() -> Result<()> { @r" 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, preserve_order=true, sort_exprs=a1@0 ASC NULLS LAST - RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 - StreamingTableExec: partition_sizes=1, projection=[a1, a2], infinite_source=true, output_ordering=[a1@0 ASC NULLS LAST] + RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=1 + StreamingTableExec: partition_sizes=1, projection=[a1, a2], infinite_source=true, output_ordering=[a1@0 ASC NULLS LAST] CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a1@0 ASC NULLS LAST - RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 - StreamingTableExec: partition_sizes=1, projection=[a1, a2], infinite_source=true, output_ordering=[a1@0 ASC NULLS LAST] + RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=1 + StreamingTableExec: partition_sizes=1, projection=[a1, a2], infinite_source=true, output_ordering=[a1@0 ASC NULLS LAST] " ); Ok(()) @@ -134,13 +132,11 @@ async fn join_no_order_on_filter() -> Result<()> { @r" SymmetricHashJoinExec: mode=Partitioned, join_type=Full, on=[(a2@1, a2@1)], filter=CAST(a3@0 AS Int64) > CAST(a3@1 AS Int64) + 3 AND CAST(a3@0 AS Int64) < CAST(a3@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 - StreamingTableExec: partition_sizes=1, projection=[a1, a2, a3], infinite_source=true, output_ordering=[a1@0 ASC NULLS LAST] + RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=1 + StreamingTableExec: partition_sizes=1, projection=[a1, a2, a3], infinite_source=true, output_ordering=[a1@0 ASC NULLS LAST] CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=8 - RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 - StreamingTableExec: partition_sizes=1, projection=[a1, a2, a3], infinite_source=true, output_ordering=[a1@0 ASC NULLS LAST] + RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=1 + StreamingTableExec: partition_sizes=1, projection=[a1, a2, a3], infinite_source=true, output_ordering=[a1@0 ASC NULLS LAST] " ); Ok(()) @@ -177,13 +173,11 @@ async fn join_change_in_planner_without_sort() -> Result<()> { @r" 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 - StreamingTableExec: partition_sizes=1, projection=[a1, a2], infinite_source=true + RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=1 + StreamingTableExec: partition_sizes=1, projection=[a1, a2], infinite_source=true CoalesceBatchesExec: target_batch_size=8192 - RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=8 - RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 - StreamingTableExec: partition_sizes=1, projection=[a1, a2], infinite_source=true + RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=1 + StreamingTableExec: partition_sizes=1, projection=[a1, a2], infinite_source=true " ); Ok(()) diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index e9e28fec064f..4464c12ca7bf 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -1273,7 +1273,8 @@ pub fn ensure_distribution( child = add_merge_on_top(child); } Distribution::HashPartitioned(exprs) => { - if add_roundrobin { + // See https://github.com/apache/datafusion/issues/18341#issuecomment-3503238325 for background + if add_roundrobin && !hash_necessary { // Add round-robin repartitioning on top of the operator // to increase parallelism. child = add_roundrobin_on_top(child, target_partitions)?; diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index a5973afc0a93..692204b7b9cc 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -1180,10 +1180,9 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[median(alias1)] 05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] 06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------RepartitionExec: partitioning=Hash([alias1@0], 4), input_partitions=4 -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------AggregateExec: mode=Partial, gby=[c@0 as alias1], aggr=[] -10)------------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------RepartitionExec: partitioning=Hash([alias1@0], 4), input_partitions=1 +08)--------------AggregateExec: mode=Partial, gby=[c@0 as alias1], aggr=[] +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table t; @@ -7170,10 +7169,9 @@ physical_plan 02)--FilterExec: max(having_test.v1)@2 = 3, projection=[v1@0, v2@1] 03)----AggregateExec: mode=FinalPartitioned, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] 04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------RepartitionExec: partitioning=Hash([v1@0, v2@1], 4), input_partitions=4 -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------RepartitionExec: partitioning=Hash([v1@0, v2@1], 4), input_partitions=1 +06)----------AggregateExec: mode=Partial, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] query error diff --git a/datafusion/sqllogictest/test_files/aggregate_repartition.slt b/datafusion/sqllogictest/test_files/aggregate_repartition.slt index 27602b61e424..7612fc84d423 100644 --- a/datafusion/sqllogictest/test_files/aggregate_repartition.slt +++ b/datafusion/sqllogictest/test_files/aggregate_repartition.slt @@ -99,10 +99,9 @@ physical_plan 01)ProjectionExec: expr=[env@0 as env, count(Int64(1))@1 as count(*)] 02)--AggregateExec: mode=FinalPartitioned, gby=[env@0 as env], aggr=[count(Int64(1))] 03)----CoalesceBatchesExec: target_batch_size=8192 -04)------RepartitionExec: partitioning=Hash([env@0], 4), input_partitions=4 -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[env@0 as env], aggr=[count(Int64(1))] -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/aggregate_repartition/dim.parquet]]}, projection=[env], file_type=parquet +04)------RepartitionExec: partitioning=Hash([env@0], 4), input_partitions=1 +05)--------AggregateExec: mode=Partial, gby=[env@0 as env], aggr=[count(Int64(1))] +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/aggregate_repartition/dim.parquet]]}, projection=[env], file_type=parquet # Verify the queries actually work and return the same results query TI rowsort diff --git a/datafusion/sqllogictest/test_files/aggregates_topk.slt b/datafusion/sqllogictest/test_files/aggregates_topk.slt index cc1693843848..76f8eb954cf9 100644 --- a/datafusion/sqllogictest/test_files/aggregates_topk.slt +++ b/datafusion/sqllogictest/test_files/aggregates_topk.slt @@ -47,10 +47,9 @@ physical_plan 02)--SortExec: TopK(fetch=4), expr=[max(traces.timestamp)@1 DESC], preserve_partitioning=[true] 03)----AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] 04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 +06)----------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] query TI select * from (select trace_id, MAX(timestamp) max_ts from traces t group by trace_id) where trace_id != 'b' order by max_ts desc limit 3; @@ -111,10 +110,9 @@ physical_plan 02)--SortExec: TopK(fetch=4), expr=[max(traces.timestamp)@1 DESC], preserve_partitioning=[true] 03)----AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], lim=[4] 04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], lim=[4] -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 +06)----------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], lim=[4] +07)------------DataSourceExec: 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; @@ -128,10 +126,9 @@ physical_plan 02)--SortExec: TopK(fetch=4), expr=[min(traces.timestamp)@1 DESC], preserve_partitioning=[true] 03)----AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[min(traces.timestamp)] 04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[min(traces.timestamp)] -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 +06)----------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[min(traces.timestamp)] +07)------------DataSourceExec: 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; @@ -145,10 +142,9 @@ physical_plan 02)--SortExec: TopK(fetch=4), expr=[max(traces.timestamp)@1 ASC NULLS LAST], preserve_partitioning=[true] 03)----AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] 04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 +06)----------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] +07)------------DataSourceExec: 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; @@ -162,10 +158,9 @@ physical_plan 02)--SortExec: TopK(fetch=4), expr=[trace_id@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] 04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 +06)----------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] +07)------------DataSourceExec: 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; @@ -236,10 +231,9 @@ physical_plan 02)--SortExec: TopK(fetch=4), expr=[max(traces_utf8view.timestamp)@1 DESC], preserve_partitioning=[true] 03)----AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces_utf8view.timestamp)], lim=[4] 04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces_utf8view.timestamp)], lim=[4] -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 +06)----------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces_utf8view.timestamp)], lim=[4] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] # Also add LargeUtf8 to test PR https://github.com/apache/datafusion/pull/15152 @@ -264,10 +258,9 @@ physical_plan 02)--SortExec: TopK(fetch=4), expr=[max(traces_largeutf8.timestamp)@1 DESC], preserve_partitioning=[true] 03)----AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces_largeutf8.timestamp)], lim=[4] 04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces_largeutf8.timestamp)], lim=[4] -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 +06)----------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces_largeutf8.timestamp)], lim=[4] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok diff --git a/datafusion/sqllogictest/test_files/count_star_rule.slt b/datafusion/sqllogictest/test_files/count_star_rule.slt index b78c021a565c..19d9ddecc9ff 100644 --- a/datafusion/sqllogictest/test_files/count_star_rule.slt +++ b/datafusion/sqllogictest/test_files/count_star_rule.slt @@ -1,4 +1,4 @@ -# Licensed to the Apache Software Foundation (ASF) under one +# Licensed to the Apache Software Foundation (ASF) under onecount_star # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information # regarding copyright ownership. The ASF licenses this file @@ -50,10 +50,9 @@ physical_plan 01)ProjectionExec: expr=[a@0 as a, count(Int64(1))@1 as count()] 02)--AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] 03)----CoalesceBatchesExec: target_batch_size=8192 -04)------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +04)------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 +05)--------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] query TT EXPLAIN SELECT t1.a, COUNT() AS cnt FROM t1 GROUP BY t1.a HAVING COUNT() > 0; @@ -69,10 +68,9 @@ physical_plan 03)----FilterExec: count(Int64(1))@1 > 0 04)------AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] 05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +06)----------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 +07)------------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT t1.a, COUNT() AS cnt FROM t1 GROUP BY t1.a HAVING COUNT() > 1; diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index 22f19a0af32e..5f3c778fc961 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -1405,33 +1405,24 @@ physical_plan 34)│ RepartitionExec ││ RepartitionExec │ 35)│ -------------------- ││ -------------------- │ 36)│ partition_count(in->out): ││ partition_count(in->out): │ -37)│ 4 -> 4 ││ 4 -> 4 │ +37)│ 1 -> 4 ││ 1 -> 4 │ 38)│ ││ │ 39)│ partitioning_scheme: ││ partitioning_scheme: │ 40)│ Hash([name@0], 4) ││ Hash([name@0], 4) │ 41)└─────────────┬─────────────┘└─────────────┬─────────────┘ 42)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -43)│ RepartitionExec ││ RepartitionExec │ +43)│ AggregateExec ││ AggregateExec │ 44)│ -------------------- ││ -------------------- │ -45)│ partition_count(in->out): ││ partition_count(in->out): │ -46)│ 1 -> 4 ││ 1 -> 4 │ -47)│ ││ │ -48)│ partitioning_scheme: ││ partitioning_scheme: │ -49)│ RoundRobinBatch(4) ││ RoundRobinBatch(4) │ -50)└─────────────┬─────────────┘└─────────────┬─────────────┘ -51)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -52)│ AggregateExec ││ AggregateExec │ -53)│ -------------------- ││ -------------------- │ -54)│ group_by: name ││ group_by: name │ -55)│ mode: Partial ││ mode: Partial │ -56)└─────────────┬─────────────┘└─────────────┬─────────────┘ -57)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -58)│ DataSourceExec ││ DataSourceExec │ -59)│ -------------------- ││ -------------------- │ -60)│ bytes: 296 ││ bytes: 288 │ -61)│ format: memory ││ format: memory │ -62)│ rows: 1 ││ rows: 1 │ -63)└───────────────────────────┘└───────────────────────────┘ +45)│ group_by: name ││ group_by: name │ +46)│ mode: Partial ││ mode: Partial │ +47)└─────────────┬─────────────┘└─────────────┬─────────────┘ +48)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +49)│ DataSourceExec ││ DataSourceExec │ +50)│ -------------------- ││ -------------------- │ +51)│ bytes: 296 ││ bytes: 288 │ +52)│ format: memory ││ format: memory │ +53)│ rows: 1 ││ rows: 1 │ +54)└───────────────────────────┘└───────────────────────────┘ # Test explain tree for UnionExec query TT diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index fe7871c22b4c..b74815edaa57 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2989,10 +2989,9 @@ physical_plan 03)----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] 04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] 05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -08)--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=1 +07)------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] query TRR SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -3025,10 +3024,9 @@ physical_plan 03)----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] 04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] 05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -08)--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=1 +07)------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] query TRR @@ -4245,10 +4243,9 @@ physical_plan 01)ProjectionExec: expr=[sum(DISTINCT t1.x)@1 as sum(DISTINCT t1.x), max(DISTINCT t1.x)@2 as max(DISTINCT t1.x)] 02)--AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[sum(DISTINCT t1.x), max(DISTINCT t1.x)] 03)----CoalesceBatchesExec: target_batch_size=2 -04)------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 -05)--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[y@1 as y], aggr=[sum(DISTINCT t1.x), max(DISTINCT t1.x)] -07)------------DataSourceExec: partitions=1, partition_sizes=[1] +04)------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=1 +05)--------AggregateExec: mode=Partial, gby=[y@1 as y], aggr=[sum(DISTINCT t1.x), max(DISTINCT t1.x)] +06)----------DataSourceExec: 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; @@ -4266,10 +4263,9 @@ physical_plan 05)--------AggregateExec: mode=Partial, gby=[y@0 as y], aggr=[sum(alias1), max(alias1)] 06)----------AggregateExec: mode=FinalPartitioned, gby=[y@0 as y, alias1@1 as alias1], aggr=[] 07)------------CoalesceBatchesExec: target_batch_size=2 -08)--------------RepartitionExec: partitioning=Hash([y@0, alias1@1], 8), input_partitions=8 -09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -10)------------------AggregateExec: mode=Partial, gby=[y@1 as y, CAST(x@0 AS Float64) as alias1], aggr=[] -11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------RepartitionExec: partitioning=Hash([y@0, alias1@1], 8), input_partitions=1 +09)----------------AggregateExec: mode=Partial, gby=[y@1 as y, CAST(x@0 AS Float64) as alias1], aggr=[] +10)------------------DataSourceExec: 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 b8b2a7c37276..4551b6cb4975 100644 --- a/datafusion/sqllogictest/test_files/insert.slt +++ b/datafusion/sqllogictest/test_files/insert.slt @@ -71,9 +71,8 @@ physical_plan 05)--------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: Field { "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": nullable Int64 }, frame: ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, count(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Field { "count(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING": Int64 }, frame: ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING], mode=[Sorted] 06)----------SortExec: expr=[c1@0 ASC NULLS LAST, c9@2 ASC NULLS LAST], preserve_partitioning=[true] 07)------------CoalesceBatchesExec: target_batch_size=8192 -08)--------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 -09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true +08)--------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=1 +09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true query I INSERT INTO table_without_values SELECT @@ -131,9 +130,8 @@ physical_plan 04)------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: Field { "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": nullable Int64 }, frame: ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, count(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Field { "count(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING": Int64 }, frame: ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING], mode=[Sorted] 05)--------SortExec: expr=[c1@0 ASC NULLS LAST, c9@2 ASC NULLS LAST], preserve_partitioning=[true] 06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 -08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true +07)------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=1 +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true @@ -182,9 +180,8 @@ physical_plan 05)--------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: Field { "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": nullable Int64 }, frame: ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, count(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Field { "count(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING": Int64 }, frame: ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING], mode=[Sorted] 06)----------SortExec: expr=[c1@0 ASC NULLS LAST, c9@2 ASC NULLS LAST], preserve_partitioning=[true] 07)------------CoalesceBatchesExec: target_batch_size=8192 -08)--------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 -09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true +08)--------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=1 +09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true query I diff --git a/datafusion/sqllogictest/test_files/insert_to_external.slt b/datafusion/sqllogictest/test_files/insert_to_external.slt index dc8ef59bbedc..2642b2780f98 100644 --- a/datafusion/sqllogictest/test_files/insert_to_external.slt +++ b/datafusion/sqllogictest/test_files/insert_to_external.slt @@ -425,9 +425,8 @@ physical_plan 05)--------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: Field { "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": nullable Int64 }, frame: ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, count(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Field { "count(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING": Int64 }, frame: ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING], mode=[Sorted] 06)----------SortExec: expr=[c1@0 ASC NULLS LAST, c9@2 ASC NULLS LAST], preserve_partitioning=[true] 07)------------CoalesceBatchesExec: target_batch_size=8192 -08)--------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 -09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true +08)--------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=1 +09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true query I INSERT INTO table_without_values SELECT @@ -486,9 +485,8 @@ physical_plan 04)------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: Field { "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": nullable Int64 }, frame: ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, count(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Field { "count(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING": Int64 }, frame: ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING], mode=[Sorted] 05)--------SortExec: expr=[c1@0 ASC NULLS LAST, c9@2 ASC NULLS LAST], preserve_partitioning=[true] 06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 -08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true +07)------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=1 +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true diff --git a/datafusion/sqllogictest/test_files/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index fe3356af88fc..87373af1472a 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -1438,11 +1438,10 @@ physical_plan 05)--------ProjectionExec: expr=[sum(t1.v1)@1 as sum(t1.v1), v0@0 as v0] 06)----------AggregateExec: mode=FinalPartitioned, gby=[v0@0 as v0], aggr=[sum(t1.v1)] 07)------------CoalesceBatchesExec: target_batch_size=8192 -08)--------------RepartitionExec: partitioning=Hash([v0@0], 4), input_partitions=4 -09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------------AggregateExec: mode=Partial, gby=[v0@0 as v0], aggr=[sum(t1.v1)] -11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] -12)------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------RepartitionExec: partitioning=Hash([v0@0], 4), input_partitions=1 +09)----------------AggregateExec: mode=Partial, gby=[v0@0 as v0], aggr=[sum(t1.v1)] +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] +11)------DataSourceExec: partitions=1, partition_sizes=[1] query III SELECT * diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 4bdf2e5da963..62804ad76bd6 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -2746,14 +2746,12 @@ physical_plan 01)SortMergeJoin: join_type=Inner, on=[(c1@0, c1@0)] 02)--SortExec: expr=[c1@0 ASC], preserve_partitioning=[true] 03)----CoalesceBatchesExec: target_batch_size=2 -04)------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)--SortExec: expr=[c1@0 ASC], preserve_partitioning=[true] -08)----CoalesceBatchesExec: target_batch_size=2 -09)------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -10)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -11)----------DataSourceExec: partitions=1, partition_sizes=[1] +04)------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=1 +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)--SortExec: expr=[c1@0 ASC], preserve_partitioning=[true] +07)----CoalesceBatchesExec: target_batch_size=2 +08)------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=1 +09)--------DataSourceExec: partitions=1, partition_sizes=[1] # sort_merge_join_on_date32 inner sort merge join on data type (Date32) query DDRTDDRT rowsort @@ -2783,9 +2781,8 @@ physical_plan 08)--------------DataSourceExec: partitions=1, partition_sizes=[1] 09)----SortExec: expr=[c3@2 ASC], preserve_partitioning=[true] 10)------CoalesceBatchesExec: target_batch_size=2 -11)--------RepartitionExec: partitioning=Hash([c3@2], 2), input_partitions=2 -12)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -13)------------DataSourceExec: partitions=1, partition_sizes=[1] +11)--------RepartitionExec: partitioning=Hash([c3@2], 2), input_partitions=1 +12)----------DataSourceExec: partitions=1, partition_sizes=[1] # sort_merge_join_on_decimal right join on data type (Decimal) query DDRTDDRT rowsort @@ -3196,15 +3193,13 @@ physical_plan 01)SortPreservingMergeExec: [rn1@5 ASC NULLS LAST] 02)--SortMergeJoin: join_type=Inner, on=[(a@1, a@1)] 03)----CoalesceBatchesExec: target_batch_size=2 -04)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, rn1@5 ASC NULLS LAST -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)----------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] -07)------------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": UInt64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] -08)--------------DataSourceExec: 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], file_type=csv, has_header=true -09)----CoalesceBatchesExec: target_batch_size=2 -10)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST -11)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -12)----------DataSourceExec: 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], file_type=csv, has_header=true +04)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=1 +05)--------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] +06)----------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": UInt64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] +07)------------DataSourceExec: 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], file_type=csv, has_header=true +08)----CoalesceBatchesExec: target_batch_size=2 +09)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=1 +10)--------DataSourceExec: 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], file_type=csv, 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 @@ -3230,15 +3225,13 @@ physical_plan 01)SortPreservingMergeExec: [rn1@10 ASC NULLS LAST] 02)--SortMergeJoin: join_type=Right, on=[(a@1, a@1)] 03)----CoalesceBatchesExec: target_batch_size=2 -04)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)----------DataSourceExec: 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], file_type=csv, has_header=true -07)----CoalesceBatchesExec: target_batch_size=2 -08)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, rn1@5 ASC NULLS LAST -09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)----------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] -11)------------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": UInt64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] -12)--------------DataSourceExec: 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], file_type=csv, has_header=true +04)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=1 +05)--------DataSourceExec: 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], file_type=csv, has_header=true +06)----CoalesceBatchesExec: target_batch_size=2 +07)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=1 +08)--------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] +09)----------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": UInt64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] +10)------------DataSourceExec: 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], file_type=csv, has_header=true statement ok set datafusion.optimizer.prefer_existing_sort = false; @@ -3269,22 +3262,17 @@ logical_plan 10)----------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan 01)SortPreservingMergeExec: [a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, rn1@11 ASC NULLS LAST] -02)--SortExec: expr=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, rn1@11 ASC NULLS LAST], preserve_partitioning=[true] -03)----SortMergeJoin: join_type=Inner, on=[(a@1, a@1)] -04)------SortExec: expr=[a@1 ASC], preserve_partitioning=[true] -05)--------CoalesceBatchesExec: target_batch_size=2 -06)----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)--------------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] -09)----------------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": UInt64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] -10)------------------DataSourceExec: 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], file_type=csv, has_header=true -11)------SortExec: expr=[a@1 ASC], preserve_partitioning=[true] -12)--------CoalesceBatchesExec: target_batch_size=2 -13)----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -14)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -15)--------------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] -16)----------------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": UInt64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] -17)------------------DataSourceExec: 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], file_type=csv, has_header=true +02)--SortMergeJoin: join_type=Inner, on=[(a@1, a@1)] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=1 +05)--------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] +06)----------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": UInt64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] +07)------------DataSourceExec: 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], file_type=csv, has_header=true +08)----CoalesceBatchesExec: target_batch_size=2 +09)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=1 +10)--------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] +11)----------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": UInt64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] +12)------------DataSourceExec: 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], file_type=csv, has_header=true statement ok set datafusion.optimizer.prefer_hash_join = true; @@ -3471,22 +3459,19 @@ logical_plan 08)----------TableScan: annotated_data projection=[a, b] physical_plan 01)SortPreservingMergeExec: [a@0 ASC] -02)--SortExec: expr=[a@0 ASC], preserve_partitioning=[true] -03)----ProjectionExec: expr=[a@0 as a, last_value(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] -04)------AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[last_value(r.b) ORDER BY [r.a ASC NULLS FIRST]] -05)--------CoalesceBatchesExec: target_batch_size=2 -06)----------RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 2), input_partitions=2 -07)------------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[last_value(r.b) ORDER BY [r.a ASC NULLS FIRST]] -08)--------------CoalesceBatchesExec: target_batch_size=2 -09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)] -10)------------------CoalesceBatchesExec: target_batch_size=2 -11)--------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 -12)----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -13)------------------------DataSourceExec: 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], file_type=csv, has_header=true -14)------------------CoalesceBatchesExec: target_batch_size=2 -15)--------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 -16)----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -17)------------------------DataSourceExec: 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], file_type=csv, has_header=true +02)--ProjectionExec: expr=[a@0 as a, last_value(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] +03)----AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[last_value(r.b) ORDER BY [r.a ASC NULLS FIRST]], ordering_mode=PartiallySorted([0]) +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 2), input_partitions=2, preserve_order=true, sort_exprs=a@0 ASC +06)----------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[last_value(r.b) ORDER BY [r.a ASC NULLS FIRST]], ordering_mode=PartiallySorted([0]) +07)------------CoalesceBatchesExec: target_batch_size=2 +08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)] +09)----------------CoalesceBatchesExec: target_batch_size=2 +10)------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=1 +11)--------------------DataSourceExec: 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], file_type=csv, has_header=true +12)----------------CoalesceBatchesExec: target_batch_size=2 +13)------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=1 +14)--------------------DataSourceExec: 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], file_type=csv, has_header=true query TT EXPLAIN SELECT * diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index ae82aee5e155..25b741b025a7 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -406,10 +406,9 @@ logical_plan physical_plan 01)AggregateExec: mode=FinalPartitioned, gby=[i@0 as i], aggr=[] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----RepartitionExec: partitioning=Hash([i@0], 4), input_partitions=4 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------AggregateExec: mode=Partial, gby=[i@0 as i], aggr=[] -06)----------DataSourceExec: partitions=1 +03)----RepartitionExec: partitioning=Hash([i@0], 4), input_partitions=1 +04)------AggregateExec: mode=Partial, gby=[i@0 as i], aggr=[] +05)--------DataSourceExec: partitions=1 statement ok set datafusion.explain.show_sizes = true; diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index 04a7615c764b..f118644a43d7 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -894,19 +894,17 @@ physical_plan 04)------ProjectionExec: expr=[0 as m, t@0 as t] 05)--------AggregateExec: mode=FinalPartitioned, gby=[t@0 as t], aggr=[] 06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------RepartitionExec: partitioning=Hash([t@0], 2), input_partitions=2 -08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -09)----------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[] -10)------------------ProjectionExec: expr=[column1@0 as t] -11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] -12)------ProjectionExec: expr=[1 as m, t@0 as t] -13)--------AggregateExec: mode=FinalPartitioned, gby=[t@0 as t], aggr=[] -14)----------CoalesceBatchesExec: target_batch_size=8192 -15)------------RepartitionExec: partitioning=Hash([t@0], 2), input_partitions=2 -16)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -17)----------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[] -18)------------------ProjectionExec: expr=[column1@0 as t] -19)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +07)------------RepartitionExec: partitioning=Hash([t@0], 2), input_partitions=1 +08)--------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[] +09)----------------ProjectionExec: expr=[column1@0 as t] +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] +11)------ProjectionExec: expr=[1 as m, t@0 as t] +12)--------AggregateExec: mode=FinalPartitioned, gby=[t@0 as t], aggr=[] +13)----------CoalesceBatchesExec: target_batch_size=8192 +14)------------RepartitionExec: partitioning=Hash([t@0], 2), input_partitions=1 +15)--------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[] +16)----------------ProjectionExec: expr=[column1@0 as t] +17)------------------DataSourceExec: partitions=1, partition_sizes=[1] ##### # Multi column sorting with lists diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index 77ee3e4f05a0..c10e67a22535 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -767,15 +767,14 @@ physical_plan 06)--------CoalesceBatchesExec: target_batch_size=8192 07)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_partkey@3] 08)------------CoalesceBatchesExec: target_batch_size=8192 -09)--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -10)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_extendedprice, l_discount], file_type=csv, has_header=true -12)------------CoalesceBatchesExec: target_batch_size=8192 -13)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -14)----------------CoalesceBatchesExec: target_batch_size=8192 -15)------------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23, projection=[p_partkey@0] -16)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -17)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], file_type=csv, has_header=true +09)--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=1 +10)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_extendedprice, l_discount], file_type=csv, has_header=true +11)------------CoalesceBatchesExec: target_batch_size=8192 +12)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +13)----------------CoalesceBatchesExec: target_batch_size=8192 +14)------------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23, projection=[p_partkey@0] +15)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +16)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], file_type=csv, has_header=true # Simplification of a binary operator with a NULL value diff --git a/datafusion/sqllogictest/test_files/qualify.slt b/datafusion/sqllogictest/test_files/qualify.slt index 366d65df6792..524f6baad2be 100644 --- a/datafusion/sqllogictest/test_files/qualify.slt +++ b/datafusion/sqllogictest/test_files/qualify.slt @@ -363,10 +363,9 @@ physical_plan 09)----------------SortExec: expr=[sum(users.salary)@1 DESC], preserve_partitioning=[true] 10)------------------AggregateExec: mode=FinalPartitioned, gby=[dept@0 as dept], aggr=[sum(users.salary)] 11)--------------------CoalesceBatchesExec: target_batch_size=8192 -12)----------------------RepartitionExec: partitioning=Hash([dept@0], 4), input_partitions=4 -13)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -14)--------------------------AggregateExec: mode=Partial, gby=[dept@1 as dept], aggr=[sum(users.salary)] -15)----------------------------DataSourceExec: partitions=1, partition_sizes=[1] +12)----------------------RepartitionExec: partitioning=Hash([dept@0], 4), input_partitions=1 +13)------------------------AggregateExec: mode=Partial, gby=[dept@1 as dept], aggr=[sum(users.salary)] +14)--------------------------DataSourceExec: partitions=1, partition_sizes=[1] # Clean up statement ok diff --git a/datafusion/sqllogictest/test_files/repartition.slt b/datafusion/sqllogictest/test_files/repartition.slt index 29d20d10b671..a3b6b380c57f 100644 --- a/datafusion/sqllogictest/test_files/repartition.slt +++ b/datafusion/sqllogictest/test_files/repartition.slt @@ -45,10 +45,9 @@ logical_plan physical_plan 01)AggregateExec: mode=FinalPartitioned, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=4 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2], file_type=parquet +03)----RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=1 +04)------AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2], file_type=parquet # disable round robin repartitioning statement ok diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part index 04de9153a047..4cfd69bbc24f 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part @@ -87,20 +87,18 @@ physical_plan 16)------------------------------CoalesceBatchesExec: target_batch_size=8192 17)--------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, o_orderkey@7] 18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -19)------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -20)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -21)----------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -22)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -23)------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -24)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -25)----------------------------------------FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1] -26)------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -27)--------------------------CoalesceBatchesExec: target_batch_size=8192 -28)----------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -29)------------------------------CoalesceBatchesExec: target_batch_size=8192 -30)--------------------------------FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] -31)----------------------------------DataSourceExec: 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], file_type=csv, has_header=false -32)------------------CoalesceBatchesExec: target_batch_size=8192 -33)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -34)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -35)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +19)------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 +20)--------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +21)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +22)------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +23)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +24)----------------------------------------FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1] +25)------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +26)--------------------------CoalesceBatchesExec: target_batch_size=8192 +27)----------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +28)------------------------------CoalesceBatchesExec: target_batch_size=8192 +29)--------------------------------FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] +30)----------------------------------DataSourceExec: 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], file_type=csv, has_header=false +31)------------------CoalesceBatchesExec: target_batch_size=8192 +32)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +33)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part index 6b03d708c7fa..52bbd7b63afb 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part @@ -92,35 +92,33 @@ physical_plan 17)--------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 18)----------------------------------DataSourceExec: 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], file_type=csv, has_header=false 19)------------------------------CoalesceBatchesExec: target_batch_size=8192 -20)--------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -21)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -22)------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -23)----------------------CoalesceBatchesExec: target_batch_size=8192 -24)------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -25)--------------------------CoalesceBatchesExec: target_batch_size=8192 -26)----------------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] -27)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -28)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false -29)------ProjectionExec: expr=[CAST(CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] -30)--------AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] -31)----------CoalescePartitionsExec -32)------------AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] -33)--------------CoalesceBatchesExec: target_batch_size=8192 -34)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] -35)------------------CoalesceBatchesExec: target_batch_size=8192 -36)--------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -37)----------------------CoalesceBatchesExec: target_batch_size=8192 -38)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] -39)--------------------------CoalesceBatchesExec: target_batch_size=8192 -40)----------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -41)------------------------------DataSourceExec: 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], file_type=csv, has_header=false -42)--------------------------CoalesceBatchesExec: target_batch_size=8192 -43)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -44)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -45)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -46)------------------CoalesceBatchesExec: target_batch_size=8192 -47)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -48)----------------------CoalesceBatchesExec: target_batch_size=8192 -49)------------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] -50)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -51)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +20)--------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +21)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +22)----------------------CoalesceBatchesExec: target_batch_size=8192 +23)------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +24)--------------------------CoalesceBatchesExec: target_batch_size=8192 +25)----------------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] +26)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +27)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +28)------ProjectionExec: expr=[CAST(CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] +29)--------AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +30)----------CoalescePartitionsExec +31)------------AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +32)--------------CoalesceBatchesExec: target_batch_size=8192 +33)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] +34)------------------CoalesceBatchesExec: target_batch_size=8192 +35)--------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +36)----------------------CoalesceBatchesExec: target_batch_size=8192 +37)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] +38)--------------------------CoalesceBatchesExec: target_batch_size=8192 +39)----------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +40)------------------------------DataSourceExec: 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], file_type=csv, has_header=false +41)--------------------------CoalesceBatchesExec: target_batch_size=8192 +42)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +43)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +44)------------------CoalesceBatchesExec: target_batch_size=8192 +45)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +46)----------------------CoalesceBatchesExec: target_batch_size=8192 +47)------------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] +48)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +49)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part index 96f3bd6edf32..17d827cebb82 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part @@ -65,11 +65,10 @@ physical_plan 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, o_orderkey@1] 12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -15)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], file_type=csv, has_header=false -16)----------------------CoalesceBatchesExec: target_batch_size=8192 -17)------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -18)--------------------------CoalesceBatchesExec: target_batch_size=8192 -19)----------------------------FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] -20)------------------------------DataSourceExec: 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_comment], file_type=csv, has_header=false +13)------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 +14)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], file_type=csv, has_header=false +15)----------------------CoalesceBatchesExec: target_batch_size=8192 +16)------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +17)--------------------------CoalesceBatchesExec: target_batch_size=8192 +18)----------------------------FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] +19)------------------------------DataSourceExec: 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_comment], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part index 8d8dd68c3d7b..71dea1a5e12a 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part @@ -54,6 +54,5 @@ physical_plan 11)--------------------FilterExec: l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01, projection=[l_partkey@0, l_extendedprice@1, l_discount@2] 12)----------------------DataSourceExec: 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], file_type=csv, has_header=false 13)--------------CoalesceBatchesExec: target_batch_size=8192 -14)----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -15)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -16)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], file_type=csv, has_header=false +14)----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=1 +15)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part index 0636a033b25a..a3284b484122 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part @@ -89,14 +89,13 @@ physical_plan 16)--------CoalesceBatchesExec: target_batch_size=8192 17)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, supplier_no@0)], projection=[s_suppkey@0, s_name@1, s_address@2, s_phone@3, total_revenue@5] 18)------------CoalesceBatchesExec: target_batch_size=8192 -19)--------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -20)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -21)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_phone], file_type=csv, has_header=false -22)------------ProjectionExec: expr=[l_suppkey@0 as supplier_no, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] -23)--------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -24)----------------CoalesceBatchesExec: target_batch_size=8192 -25)------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 -26)--------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -27)----------------------CoalesceBatchesExec: target_batch_size=8192 -28)------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] -29)--------------------------DataSourceExec: 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_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false +19)--------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +20)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_phone], file_type=csv, has_header=false +21)------------ProjectionExec: expr=[l_suppkey@0 as supplier_no, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] +22)--------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +23)----------------CoalesceBatchesExec: target_batch_size=8192 +24)------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 +25)--------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +26)----------------------CoalesceBatchesExec: target_batch_size=8192 +27)------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] +28)--------------------------DataSourceExec: 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_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part index 55da5371671e..16a5b7eb39b5 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part @@ -82,19 +82,18 @@ physical_plan 13)------------------------CoalesceBatchesExec: target_batch_size=8192 14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@4, o_orderdate@5] 15)----------------------------CoalesceBatchesExec: target_batch_size=8192 -16)------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -17)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -18)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name], file_type=csv, has_header=false -19)----------------------------CoalesceBatchesExec: target_batch_size=8192 -20)------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -21)--------------------------------DataSourceExec: 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], file_type=csv, has_header=false -22)--------------------CoalesceBatchesExec: target_batch_size=8192 -23)----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -24)------------------------DataSourceExec: 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_quantity], file_type=csv, has_header=false -25)----------------CoalesceBatchesExec: target_batch_size=8192 -26)------------------FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0] -27)--------------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] -28)----------------------CoalesceBatchesExec: target_batch_size=8192 -29)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -30)--------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] -31)----------------------------DataSourceExec: 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_quantity], file_type=csv, has_header=false +16)------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 +17)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name], file_type=csv, has_header=false +18)----------------------------CoalesceBatchesExec: target_batch_size=8192 +19)------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +20)--------------------------------DataSourceExec: 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], file_type=csv, has_header=false +21)--------------------CoalesceBatchesExec: target_batch_size=8192 +22)----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +23)------------------------DataSourceExec: 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_quantity], file_type=csv, has_header=false +24)----------------CoalesceBatchesExec: target_batch_size=8192 +25)------------------FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0] +26)--------------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] +27)----------------------CoalesceBatchesExec: target_batch_size=8192 +28)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +29)--------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] +30)----------------------------DataSourceExec: 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_quantity], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part index b2e0fb0cd1cc..c299fa88a9c4 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part @@ -131,50 +131,46 @@ physical_plan 30)----------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 31)------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false 32)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -33)--------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -34)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -35)------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -36)--------------------------CoalesceBatchesExec: target_batch_size=8192 -37)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -38)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -39)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], file_type=csv, has_header=false -40)------------------CoalesceBatchesExec: target_batch_size=8192 -41)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -42)----------------------CoalesceBatchesExec: target_batch_size=8192 -43)------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] -44)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -45)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false -46)----------CoalesceBatchesExec: target_batch_size=8192 -47)------------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 -48)--------------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] -49)----------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] -50)------------------CoalesceBatchesExec: target_batch_size=8192 -51)--------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -52)----------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] -53)------------------------CoalesceBatchesExec: target_batch_size=8192 -54)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] -55)----------------------------CoalesceBatchesExec: target_batch_size=8192 -56)------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 -57)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -58)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] -59)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -60)--------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -61)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -62)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] -63)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -64)----------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 -65)------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -66)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -67)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -68)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -69)--------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -70)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -71)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -72)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -73)------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], file_type=csv, has_header=false -74)----------------------------CoalesceBatchesExec: target_batch_size=8192 -75)------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -76)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -77)----------------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] -78)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -79)--------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false +33)--------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +34)----------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +35)--------------------------CoalesceBatchesExec: target_batch_size=8192 +36)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +37)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], file_type=csv, has_header=false +38)------------------CoalesceBatchesExec: target_batch_size=8192 +39)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +40)----------------------CoalesceBatchesExec: target_batch_size=8192 +41)------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] +42)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +43)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false +44)----------CoalesceBatchesExec: target_batch_size=8192 +45)------------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 +46)--------------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] +47)----------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +48)------------------CoalesceBatchesExec: target_batch_size=8192 +49)--------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +50)----------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +51)------------------------CoalesceBatchesExec: target_batch_size=8192 +52)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] +53)----------------------------CoalesceBatchesExec: target_batch_size=8192 +54)------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 +55)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +56)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] +57)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +58)--------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +59)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +60)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] +61)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +62)----------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 +63)------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +64)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +65)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +66)------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +67)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +68)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +69)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], file_type=csv, has_header=false +70)----------------------------CoalesceBatchesExec: target_batch_size=8192 +71)------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +72)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +73)----------------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] +74)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +75)--------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part index 0b994de411ea..492c68d6aaa0 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part @@ -90,37 +90,36 @@ physical_plan 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[s_suppkey@0, s_name@1, s_address@2] 09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 -11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey], file_type=csv, has_header=false -13)----------------CoalesceBatchesExec: target_batch_size=8192 -14)------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -15)--------------------CoalesceBatchesExec: target_batch_size=8192 -16)----------------------FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0] -17)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -18)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false -19)--------CoalesceBatchesExec: target_batch_size=8192 -20)----------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -21)------------CoalesceBatchesExec: target_batch_size=8192 -22)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * sum(lineitem.l_quantity)@1, projection=[ps_suppkey@1] -23)----------------CoalesceBatchesExec: target_batch_size=8192 -24)------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 -25)--------------------CoalesceBatchesExec: target_batch_size=8192 -26)----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] -27)------------------------CoalesceBatchesExec: target_batch_size=8192 -28)--------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -29)----------------------------DataSourceExec: 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], file_type=csv, has_header=false -30)------------------------CoalesceBatchesExec: target_batch_size=8192 -31)--------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -32)----------------------------CoalesceBatchesExec: target_batch_size=8192 -33)------------------------------FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] -34)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -35)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], file_type=csv, has_header=false -36)----------------ProjectionExec: expr=[0.5 * CAST(sum(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * sum(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey] -37)------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] -38)--------------------CoalesceBatchesExec: target_batch_size=8192 -39)----------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 -40)------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] -41)--------------------------CoalesceBatchesExec: target_batch_size=8192 -42)----------------------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2] -43)------------------------------DataSourceExec: 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_suppkey, l_quantity, l_shipdate], file_type=csv, has_header=false +10)------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=1 +11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey], file_type=csv, has_header=false +12)----------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +14)--------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0] +16)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +17)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +18)--------CoalesceBatchesExec: target_batch_size=8192 +19)----------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +20)------------CoalesceBatchesExec: target_batch_size=8192 +21)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * sum(lineitem.l_quantity)@1, projection=[ps_suppkey@1] +22)----------------CoalesceBatchesExec: target_batch_size=8192 +23)------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 +24)--------------------CoalesceBatchesExec: target_batch_size=8192 +25)----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] +26)------------------------CoalesceBatchesExec: target_batch_size=8192 +27)--------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +28)----------------------------DataSourceExec: 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], file_type=csv, has_header=false +29)------------------------CoalesceBatchesExec: target_batch_size=8192 +30)--------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +31)----------------------------CoalesceBatchesExec: target_batch_size=8192 +32)------------------------------FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] +33)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +34)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], file_type=csv, has_header=false +35)----------------ProjectionExec: expr=[0.5 * CAST(sum(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * sum(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey] +36)------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] +37)--------------------CoalesceBatchesExec: target_batch_size=8192 +38)----------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 +39)------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] +40)--------------------------CoalesceBatchesExec: target_batch_size=8192 +41)----------------------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2] +42)------------------------------DataSourceExec: 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_suppkey, l_quantity, l_shipdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part index e52171524007..96341ba32311 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part @@ -114,30 +114,29 @@ physical_plan 22)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 23)--------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] 24)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -25)------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -26)--------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -27)----------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_nationkey], file_type=csv, has_header=false -28)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -29)------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -30)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -31)----------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] -32)------------------------------------------------------DataSourceExec: 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_commitdate, l_receiptdate], file_type=csv, has_header=false -33)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -34)----------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -35)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -36)--------------------------------------------FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] -37)----------------------------------------------DataSourceExec: 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_orderstatus], file_type=csv, has_header=false -38)------------------------------CoalesceBatchesExec: target_batch_size=8192 -39)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -40)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -41)------------------------------------FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0] -42)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -43)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false -44)----------------------CoalesceBatchesExec: target_batch_size=8192 -45)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -46)--------------------------DataSourceExec: 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], file_type=csv, has_header=false -47)------------------CoalesceBatchesExec: target_batch_size=8192 -48)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -49)----------------------CoalesceBatchesExec: target_batch_size=8192 -50)------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] -51)--------------------------DataSourceExec: 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_commitdate, l_receiptdate], file_type=csv, has_header=false +25)------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +26)--------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_nationkey], file_type=csv, has_header=false +27)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +28)------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 +29)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +30)----------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] +31)------------------------------------------------------DataSourceExec: 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_commitdate, l_receiptdate], file_type=csv, has_header=false +32)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +33)----------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +34)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +35)--------------------------------------------FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] +36)----------------------------------------------DataSourceExec: 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_orderstatus], file_type=csv, has_header=false +37)------------------------------CoalesceBatchesExec: target_batch_size=8192 +38)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +39)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +40)------------------------------------FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0] +41)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +42)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +43)----------------------CoalesceBatchesExec: target_batch_size=8192 +44)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +45)--------------------------DataSourceExec: 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], file_type=csv, has_header=false +46)------------------CoalesceBatchesExec: target_batch_size=8192 +47)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +48)----------------------CoalesceBatchesExec: target_batch_size=8192 +49)------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] +50)--------------------------DataSourceExec: 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_commitdate, l_receiptdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part index 15636056b871..dcf462915899 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part @@ -93,28 +93,25 @@ physical_plan 24)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 25)------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_nationkey@1, o_orderkey@2] 26)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -27)----------------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -28)------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -29)--------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, has_header=false -30)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -31)----------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -32)------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -33)--------------------------------------------------------FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1] -34)----------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -35)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -36)--------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -37)----------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -38)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -39)------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=4 -40)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -41)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -42)--------------------------CoalesceBatchesExec: target_batch_size=8192 -43)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -44)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -45)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], file_type=csv, has_header=false -46)------------------CoalesceBatchesExec: target_batch_size=8192 -47)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -48)----------------------CoalesceBatchesExec: target_batch_size=8192 -49)------------------------FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] -50)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -51)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false +27)----------------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 +28)------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, has_header=false +29)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +30)----------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +31)------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +32)--------------------------------------------------------FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1] +33)----------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +34)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +35)--------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +36)----------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +37)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +38)------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=1 +39)--------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +40)--------------------------CoalesceBatchesExec: target_batch_size=8192 +41)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +42)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], file_type=csv, has_header=false +43)------------------CoalesceBatchesExec: target_batch_size=8192 +44)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +45)----------------------CoalesceBatchesExec: target_batch_size=8192 +46)------------------------FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] +47)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +48)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part index 291d56e43f2d..53ab43ba491b 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part @@ -111,30 +111,28 @@ physical_plan 25)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 26)--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5, l_shipdate@6] 27)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -28)------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -29)--------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -30)----------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -31)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -32)------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -33)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -34)----------------------------------------------------------FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 -35)------------------------------------------------------------DataSourceExec: 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, l_shipdate], file_type=csv, has_header=false -36)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -37)----------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -38)------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -39)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -40)--------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -41)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -42)------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, has_header=false -43)----------------------------CoalesceBatchesExec: target_batch_size=8192 -44)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -45)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -46)----------------------------------FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY -47)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -48)--------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false -49)--------------------CoalesceBatchesExec: target_batch_size=8192 -50)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -51)------------------------CoalesceBatchesExec: target_batch_size=8192 -52)--------------------------FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE -53)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -54)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +28)------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +29)--------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +30)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +31)------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 +32)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +33)----------------------------------------------------------FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 +34)------------------------------------------------------------DataSourceExec: 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, l_shipdate], file_type=csv, has_header=false +35)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +36)----------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +37)------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +38)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +39)--------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 +40)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, has_header=false +41)----------------------------CoalesceBatchesExec: target_batch_size=8192 +42)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +43)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +44)----------------------------------FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY +45)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +46)--------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +47)--------------------CoalesceBatchesExec: target_batch_size=8192 +48)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +49)------------------------CoalesceBatchesExec: target_batch_size=8192 +50)--------------------------FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE +51)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +52)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part index 50171c528db6..6b0b05f2f636 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part @@ -134,29 +134,25 @@ physical_plan 42)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 43)------------------------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false 44)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -45)--------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -46)----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -47)------------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -48)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -49)------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -50)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -51)----------------------------------------------------------FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 -52)------------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -53)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -54)----------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -55)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -56)--------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, has_header=false -57)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -58)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -59)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -60)------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], file_type=csv, has_header=false -61)----------------------------CoalesceBatchesExec: target_batch_size=8192 -62)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -63)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -64)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false -65)--------------------CoalesceBatchesExec: target_batch_size=8192 -66)----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -67)------------------------CoalesceBatchesExec: target_batch_size=8192 -68)--------------------------FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0] -69)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -70)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false +45)--------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +46)----------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +47)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +48)------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +49)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +50)----------------------------------------------------------FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 +51)------------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +52)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +53)----------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 +54)------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, has_header=false +55)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +56)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +57)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], file_type=csv, has_header=false +58)----------------------------CoalesceBatchesExec: target_batch_size=8192 +59)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +60)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +61)--------------------CoalesceBatchesExec: target_batch_size=8192 +62)----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +63)------------------------CoalesceBatchesExec: target_batch_size=8192 +64)--------------------------FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0] +65)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +66)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part index 3b31c1bc2e8e..dcee31dfecd3 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part @@ -111,16 +111,14 @@ physical_plan 34)------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 35)--------------------------------------------------------DataSourceExec: 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], file_type=csv, has_header=false 36)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -37)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -38)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -39)--------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -40)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -41)--------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 -42)----------------------------------------DataSourceExec: 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], file_type=csv, has_header=false -43)----------------------------CoalesceBatchesExec: target_batch_size=8192 -44)------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -45)--------------------------------DataSourceExec: 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], file_type=csv, has_header=false -46)--------------------CoalesceBatchesExec: target_batch_size=8192 -47)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -48)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -49)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +37)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +38)------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +39)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +40)--------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 +41)----------------------------------------DataSourceExec: 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], file_type=csv, has_header=false +42)----------------------------CoalesceBatchesExec: target_batch_size=8192 +43)------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +44)--------------------------------DataSourceExec: 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], file_type=csv, has_header=false +45)--------------------CoalesceBatchesExec: target_batch_size=8192 +46)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +47)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index 0c8b8c6edb1f..f7ab6a0c9281 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -235,14 +235,13 @@ logical_plan physical_plan 01)AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 -05)--------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] -06)----------UnionExec -07)------------DataSourceExec: partitions=1, partition_sizes=[1] -08)------------DataSourceExec: partitions=1, partition_sizes=[1] -09)------------ProjectionExec: expr=[name@0 || _new as name] -10)--------------DataSourceExec: partitions=1, partition_sizes=[1] +03)----RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=3 +04)------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] +05)--------UnionExec +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)----------DataSourceExec: partitions=1, partition_sizes=[1] +08)----------ProjectionExec: expr=[name@0 || _new as name] +09)------------DataSourceExec: partitions=1, partition_sizes=[1] # nested_union_all query T rowsort diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index c2fabb5e6eff..26cb71acbdfe 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1319,9 +1319,8 @@ physical_plan 07)------------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: Field { "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": nullable Int64 }, frame: ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING], mode=[Sorted] 08)--------------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[true] 09)----------------CoalesceBatchesExec: target_batch_size=4096 -10)------------------RepartitionExec: partitioning=Hash([c1@0, c2@1], 2), input_partitions=2 -11)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c4], file_type=csv, has_header=true +10)------------------RepartitionExec: partitioning=Hash([c1@0, c2@1], 2), input_partitions=1 +11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c4], file_type=csv, has_header=true # test_window_agg_sort_reversed_plan @@ -1925,16 +1924,15 @@ logical_plan 06)----------TableScan: aggregate_test_100 projection=[c2, c3, c9] physical_plan 01)SortPreservingMergeExec: [c3@0 ASC NULLS LAST], fetch=5 -02)--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] -03)----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: Field { "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": nullable UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] -04)------SortExec: expr=[c3@0 ASC NULLS LAST, c9@1 DESC], preserve_partitioning=[true] +02)--SortExec: TopK(fetch=5), expr=[c3@0 ASC NULLS LAST], preserve_partitioning=[true] +03)----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] +04)------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: Field { "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": nullable UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] 05)--------CoalesceBatchesExec: target_batch_size=4096 -06)----------RepartitionExec: partitioning=Hash([c3@0], 2), input_partitions=2 -07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)--------------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] -09)----------------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: Field { "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": nullable UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] -10)------------------SortExec: expr=[c3@1 DESC, c9@2 DESC, c2@0 ASC NULLS LAST], preserve_partitioning=[false] -11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c9], file_type=csv, has_header=true +06)----------RepartitionExec: partitioning=Hash([c3@0], 2), input_partitions=1 +07)------------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] +08)--------------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: Field { "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": nullable UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] +09)----------------SortExec: expr=[c3@1 DESC, c9@2 DESC, c2@0 ASC NULLS LAST], preserve_partitioning=[false] +10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c9], file_type=csv, has_header=true @@ -1943,7 +1941,7 @@ SELECT c3, SUM(c9) OVER(ORDER BY c3 DESC, c9 DESC, c2 ASC) as sum1, SUM(c9) OVER(PARTITION BY c3 ORDER BY c9 DESC ) as sum2 FROM aggregate_test_100 - ORDER BY c3 + ORDER BY c3, c9 DESC LIMIT 5 ---- -117 219796664156 3023531799 @@ -1971,9 +1969,8 @@ physical_plan 03)----BoundedWindowAggExec: wdw=[row_number() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "row_number() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": UInt64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] 04)------SortExec: expr=[c1@0 ASC NULLS LAST], preserve_partitioning=[true] 05)--------CoalesceBatchesExec: target_batch_size=4096 -06)----------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true +06)----------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=1 +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true query TI SELECT c1, ROW_NUMBER() OVER (PARTITION BY c1) as rn1 FROM aggregate_test_100 ORDER BY c1 ASC @@ -2100,9 +2097,8 @@ physical_plan 03)----BoundedWindowAggExec: wdw=[row_number() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "row_number() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": UInt64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] 04)------SortExec: expr=[c1@0 ASC NULLS LAST], preserve_partitioning=[true] 05)--------CoalesceBatchesExec: target_batch_size=4096 -06)----------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true +06)----------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=1 +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true statement ok set datafusion.optimizer.repartition_sorts = true; @@ -2129,9 +2125,8 @@ physical_plan 06)----------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: Field { "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": nullable UInt64 }, frame: ROWS BETWEEN 1 PRECEDING AND 3 FOLLOWING], mode=[Sorted] 07)------------SortExec: expr=[c1@0 ASC NULLS LAST, c9@1 ASC NULLS LAST], preserve_partitioning=[true] 08)--------------CoalesceBatchesExec: target_batch_size=4096 -09)----------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -10)------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], file_type=csv, has_header=true +09)----------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=1 +10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], file_type=csv, has_header=true # test_window_agg_with_global_limit statement ok @@ -3726,9 +3721,8 @@ physical_plan 02)--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] 03)----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: Field { "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": nullable Float64 }, frame: RANGE BETWEEN 10 PRECEDING AND CURRENT ROW], mode=[Linear] 04)------CoalesceBatchesExec: target_batch_size=4096 -05)--------RepartitionExec: partitioning=Hash([d@4], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST -06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]] +05)--------RepartitionExec: partitioning=Hash([d@4], 2), input_partitions=1 +06)----------StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]] # CTAS with NTILE function statement ok @@ -4221,11 +4215,10 @@ physical_plan 01)ProjectionExec: expr=[count(Int64(1)) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as count(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 02)--BoundedWindowAggExec: wdw=[count(Int64(1)) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "count(Int64(1)) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": Int64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] 03)----CoalesceBatchesExec: target_batch_size=4096 -04)------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)----------CoalesceBatchesExec: target_batch_size=4096 -07)------------FilterExec: a@0 = 1 -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +04)------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=1 +05)--------CoalesceBatchesExec: target_batch_size=4096 +06)----------FilterExec: a@0 = 1 +07)------------DataSourceExec: partitions=1, partition_sizes=[1] query I select ROW_NUMBER() over (partition by a) from (select * from a where a = 1); @@ -4244,11 +4237,10 @@ physical_plan 01)ProjectionExec: expr=[row_number() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as row_number() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] 02)--BoundedWindowAggExec: wdw=[row_number() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "row_number() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": UInt64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] 03)----CoalesceBatchesExec: target_batch_size=4096 -04)------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)----------CoalesceBatchesExec: target_batch_size=4096 -07)------------FilterExec: a@0 = 1 -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +04)------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=1 +05)--------CoalesceBatchesExec: target_batch_size=4096 +06)----------FilterExec: a@0 = 1 +07)------------DataSourceExec: partitions=1, partition_sizes=[1] # LAG window function IGNORE/RESPECT NULLS support with ascending order and default offset 1 query TTTTTT @@ -5358,9 +5350,8 @@ physical_plan 05)--------BoundedWindowAggExec: wdw=[rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] 06)----------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[true] 07)------------CoalesceBatchesExec: target_batch_size=1 -08)--------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)------------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=1 +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] query III select c1, c2, rank @@ -5441,9 +5432,8 @@ physical_plan 05)--------BoundedWindowAggExec: wdw=[rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] 06)----------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[true] 07)------------CoalesceBatchesExec: target_batch_size=1 -08)--------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)------------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=1 +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] query III select c1, c2, rank @@ -5541,9 +5531,8 @@ physical_plan 10)------------------BoundedWindowAggExec: wdw=[rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [t1.c1] ORDER BY [t1.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] 11)--------------------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[true] 12)----------------------CoalesceBatchesExec: target_batch_size=1 -13)------------------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -15)----------------------------DataSourceExec: partitions=1, partition_sizes=[1] +13)------------------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=1 +14)--------------------------DataSourceExec: partitions=1, partition_sizes=[1] query IIII select c1, c2, rank1, rank2 @@ -5601,9 +5590,8 @@ physical_plan 02)--ProjectionExec: expr=[c1@0 as c1, sum(aggregate_test_100_ordered.c9) PARTITION BY [aggregate_test_100_ordered.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 as sum_c9] 03)----WindowAggExec: wdw=[sum(aggregate_test_100_ordered.c9) PARTITION BY [aggregate_test_100_ordered.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(aggregate_test_100_ordered.c9) PARTITION BY [aggregate_test_100_ordered.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] 04)------CoalesceBatchesExec: target_batch_size=1 -05)--------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2, preserve_order=true, sort_exprs=c1@0 ASC NULLS LAST -06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true +05)--------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=1 +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true query TT EXPLAIN SELECT SUM(c9) OVER() as sum_c9 FROM aggregate_test_100_ordered ORDER BY sum_c9; @@ -5632,9 +5620,8 @@ physical_plan 02)--ProjectionExec: expr=[c1@0 as c1, min(aggregate_test_100_ordered.c5) PARTITION BY [aggregate_test_100_ordered.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 as min_c5] 03)----WindowAggExec: wdw=[min(aggregate_test_100_ordered.c5) PARTITION BY [aggregate_test_100_ordered.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "min(aggregate_test_100_ordered.c5) PARTITION BY [aggregate_test_100_ordered.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] 04)------CoalesceBatchesExec: target_batch_size=1 -05)--------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2, preserve_order=true, sort_exprs=c1@0 ASC NULLS LAST -06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c5], output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true +05)--------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=1 +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c5], output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true query TT EXPLAIN SELECT MAX(c5) OVER() as max_c5 FROM aggregate_test_100_ordered ORDER BY max_c5; diff --git a/datafusion/sqllogictest/test_files/window_limits.slt b/datafusion/sqllogictest/test_files/window_limits.slt index 883cd4404f4f..8729e5ed412e 100644 --- a/datafusion/sqllogictest/test_files/window_limits.slt +++ b/datafusion/sqllogictest/test_files/window_limits.slt @@ -544,9 +544,8 @@ physical_plan 03)----BoundedWindowAggExec: wdw=[sum(employees.salary) PARTITION BY [employees.depname] ORDER BY [employees.empno ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND CURRENT ROW: Field { "sum(employees.salary) PARTITION BY [employees.depname] ORDER BY [employees.empno ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND CURRENT ROW": nullable UInt64 }, frame: ROWS BETWEEN 1 PRECEDING AND CURRENT ROW], mode=[Sorted] 04)------SortExec: expr=[depname@0 ASC NULLS LAST, empno@1 ASC NULLS LAST], preserve_partitioning=[true] 05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([depname@0], 4), input_partitions=4 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100_with_dates.csv]]}, projection=[depname, empno, salary], file_type=csv, has_header=true +06)----------RepartitionExec: partitioning=Hash([depname@0], 4), input_partitions=1 +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100_with_dates.csv]]}, projection=[depname, empno, salary], file_type=csv, has_header=true # should handle partition by optimized statement ok @@ -590,9 +589,8 @@ physical_plan 03)----BoundedWindowAggExec: wdw=[sum(employees.salary) PARTITION BY [employees.depname] ORDER BY [employees.empno ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND CURRENT ROW: Field { "sum(employees.salary) PARTITION BY [employees.depname] ORDER BY [employees.empno ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND CURRENT ROW": nullable UInt64 }, frame: ROWS BETWEEN 1 PRECEDING AND CURRENT ROW], mode=[Sorted] 04)------SortExec: TopK(fetch=5), expr=[depname@0 ASC NULLS LAST, empno@1 ASC NULLS LAST], preserve_partitioning=[true] 05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([depname@0], 4), input_partitions=4 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100_with_dates.csv]]}, projection=[depname, empno, salary], file_type=csv, has_header=true +06)----------RepartitionExec: partitioning=Hash([depname@0], 4), input_partitions=1 +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100_with_dates.csv]]}, projection=[depname, empno, salary], file_type=csv, has_header=true # unbounded following statement ok diff --git a/docs/source/user-guide/sql/explain.md b/docs/source/user-guide/sql/explain.md index 1caadcc29141..23101632625b 100644 --- a/docs/source/user-guide/sql/explain.md +++ b/docs/source/user-guide/sql/explain.md @@ -70,19 +70,10 @@ to see the high level structure of the plan | | │ RepartitionExec │ | | | │ -------------------- │ | | | │ input_partition_count: │ | -| | │ 16 │ | -| | │ │ | -| | │ partitioning_scheme: │ | -| | │ Hash([b@0], 16) │ | -| | └─────────────┬─────────────┘ | -| | ┌─────────────┴─────────────┐ | -| | │ RepartitionExec │ | -| | │ -------------------- │ | -| | │ input_partition_count: │ | | | │ 1 │ | | | │ │ | | | │ partitioning_scheme: │ | -| | │ RoundRobinBatch(16) │ | +| | │ Hash([b@0], 16) │ | | | └─────────────┬─────────────┘ | | | ┌─────────────┴─────────────┐ | | | │ AggregateExec │ | @@ -126,10 +117,9 @@ Elapsed 0.004 seconds. | physical_plan | ProjectionExec: expr=[sum(t.x)@1 as sum(t.x)] | | | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[sum(t.x)] | | | CoalesceBatchesExec: target_batch_size=8192 | -| | RepartitionExec: partitioning=Hash([b@0], 16), input_partitions=16 | -| | RepartitionExec: partitioning=RoundRobinBatch(16), input_partitions=1 | -| | AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[sum(t.x)] | -| | DataSourceExec: partitions=1, partition_sizes=[1] | +| | RepartitionExec: partitioning=Hash([b@0], 16), input_partitions=1 | +| | AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[sum(t.x)] | +| | DataSourceExec: partitions=1, partition_sizes=[1] | | | | +---------------+-------------------------------------------------------------------------------+ 2 row(s) fetched.