From e5cd77de66cd9bc98f8820fcdc49aa19a821e536 Mon Sep 17 00:00:00 2001 From: Gabriel Musat Mestre Date: Tue, 30 Dec 2025 10:39:05 +0100 Subject: [PATCH] Improve default task estimator --- src/distributed_planner/task_estimator.rs | 18 +- tests/clickbench_plans_test.rs | 545 +++++---- tests/tpcds_plans_test.rs | 1298 +++++++++++---------- 3 files changed, 952 insertions(+), 909 deletions(-) diff --git a/src/distributed_planner/task_estimator.rs b/src/distributed_planner/task_estimator.rs index 2bf74ab0..d8d5c283 100644 --- a/src/distributed_planner/task_estimator.rs +++ b/src/distributed_planner/task_estimator.rs @@ -6,7 +6,6 @@ use datafusion::datasource::physical_plan::FileScanConfig; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::SessionConfig; use delegate::delegate; -use std::collections::HashSet; use std::fmt::Debug; use std::sync::Arc; @@ -205,22 +204,15 @@ impl TaskEstimator for FileScanConfigTaskEstimator { let d_cfg = cfg.extensions.get::()?; - // Count how many distinct files we have in the FileScanConfig. Each file in each - // file group is a PartitionedFile rather than a full file, so it's possible that - // many entries refer to different chunks of the same physical file. By keeping a - // HashSet of the different locations of the PartitionedFiles we count how many actual - // different files we have. - let mut distinct_files = HashSet::new(); + // Count how many partitioned files we have in the FileScanConfig. + let mut partitioned_files = 0; for file_group in &file_scan.file_groups { - for file in file_group.iter() { - distinct_files.insert(file.object_meta.location.clone()); - } + partitioned_files += file_group.len(); } - let distinct_files = distinct_files.len(); // Based on the user-provided files_per_task configuration, do the math to calculate // how many tasks should be used, without surpassing the number of available workers. - let task_count = distinct_files.div_ceil(d_cfg.files_per_task); + let task_count = partitioned_files.div_ceil(d_cfg.files_per_task); Some(TaskEstimation { task_count: TaskCountAnnotation::Desired(task_count), @@ -238,7 +230,7 @@ impl TaskEstimator for FileScanConfigTaskEstimator { } // Based on the task count, attempt to scale up the partitions in the DataSourceExec by // repartitioning it. This will result in a DataSourceExec with potentially a lot of - // partitions, but as we are going wrap it with PartitionIsolatorExec that's fine. + // partitions, but as we are going to wrap it with PartitionIsolatorExec, that's fine. let dse: &DataSourceExec = plan.as_any().downcast_ref()?; let file_scan: &FileScanConfig = dse.data_source().as_any().downcast_ref()?; diff --git a/tests/clickbench_plans_test.rs b/tests/clickbench_plans_test.rs index 907c51d2..e9e07432 100644 --- a/tests/clickbench_plans_test.rs +++ b/tests/clickbench_plans_test.rs @@ -31,14 +31,14 @@ mod tests { │ ProjectionExec: expr=[count(Int64(1))@0 as count(*)] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] │ ProjectionExec: expr=[] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: AdvEngineID@0 != 0 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[AdvEngineID], file_type=parquet, predicate=AdvEngineID@0 != 0, pruning_predicate=AdvEngineID_null_count@2 != row_count@3 AND (AdvEngineID_min@0 != 0 OR 0 != AdvEngineID_max@1), required_guarantees=[AdvEngineID not in (0)] └────────────────────────────────────────────────── "); @@ -53,11 +53,11 @@ mod tests { │ ProjectionExec: expr=[sum(hits.AdvEngineID)@0 as sum(hits.AdvEngineID), count(Int64(1))@1 as count(*), avg(hits.ResolutionWidth)@2 as avg(hits.ResolutionWidth)] │ AggregateExec: mode=Final, gby=[], aggr=[sum(hits.AdvEngineID), count(Int64(1)), avg(hits.ResolutionWidth)] │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[sum(hits.AdvEngineID), count(Int64(1)), avg(hits.ResolutionWidth)] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[ResolutionWidth, AdvEngineID], file_type=parquet └────────────────────────────────────────────────── "); @@ -96,13 +96,13 @@ mod tests { ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ AggregateExec: mode=Partial, gby=[], aggr=[count(alias1)] │ AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([alias1@0], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([alias1@0], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[UserID@0 as alias1], aggr=[] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[UserID], file_type=parquet └────────────────────────────────────────────────── "); @@ -122,13 +122,13 @@ mod tests { ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ AggregateExec: mode=Partial, gby=[], aggr=[count(alias1)] │ AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([alias1@0], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([alias1@0], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[SearchPhrase@0 as alias1], aggr=[] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[SearchPhrase], file_type=parquet └────────────────────────────────────────────────── "); @@ -150,20 +150,23 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[AdvEngineID@0 as AdvEngineID, count(*)@1 as count(*)] │ SortPreservingMergeExec: [count(Int64(1))@2 DESC] - │ SortExec: expr=[count(*)@1 DESC], preserve_partitioning=[true] - │ ProjectionExec: expr=[AdvEngineID@0 as AdvEngineID, count(Int64(1))@1 as count(*), count(Int64(1))@1 as count(Int64(1))] - │ AggregateExec: mode=FinalPartitioned, gby=[AdvEngineID@0 as AdvEngineID], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([AdvEngineID@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[AdvEngineID@0 as AdvEngineID], aggr=[count(Int64(1))] - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: AdvEngineID@0 != 0 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] - │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[AdvEngineID], file_type=parquet, predicate=AdvEngineID@0 != 0, pruning_predicate=AdvEngineID_null_count@2 != row_count@3 AND (AdvEngineID_min@0 != 0 OR 0 != AdvEngineID_max@1), required_guarantees=[AdvEngineID not in (0)] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + │ SortExec: expr=[count(*)@1 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[AdvEngineID@0 as AdvEngineID, count(Int64(1))@1 as count(*), count(Int64(1))@1 as count(Int64(1))] + │ AggregateExec: mode=FinalPartitioned, gby=[AdvEngineID@0 as AdvEngineID], aggr=[count(Int64(1))] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([AdvEngineID@0], 6), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[AdvEngineID@0 as AdvEngineID], aggr=[count(Int64(1))] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: AdvEngineID@0 != 0 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[AdvEngineID], file_type=parquet, predicate=AdvEngineID@0 != 0, pruning_predicate=AdvEngineID_null_count@2 != row_count@3 AND (AdvEngineID_min@0 != 0 OR 0 != AdvEngineID_max@1), required_guarantees=[AdvEngineID not in (0)] + └────────────────────────────────────────────────── "); Ok(()) } @@ -184,13 +187,13 @@ mod tests { │ RepartitionExec: partitioning=Hash([RegionID@0], 3), input_partitions=3 │ AggregateExec: mode=Partial, gby=[RegionID@0 as RegionID], aggr=[count(alias1)] │ AggregateExec: mode=FinalPartitioned, gby=[RegionID@0 as RegionID, alias1@1 as alias1], aggr=[] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([RegionID@0, alias1@1], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([RegionID@0, alias1@1], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[RegionID@0 as RegionID, UserID@1 as alias1], aggr=[] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[RegionID, UserID], file_type=parquet └────────────────────────────────────────────────── "); @@ -209,13 +212,13 @@ mod tests { │ SortExec: TopK(fetch=10), expr=[c@2 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[RegionID@0 as RegionID, sum(hits.AdvEngineID)@1 as sum(hits.AdvEngineID), count(Int64(1))@2 as c, avg(hits.ResolutionWidth)@3 as avg(hits.ResolutionWidth), count(DISTINCT hits.UserID)@4 as count(DISTINCT hits.UserID)] │ AggregateExec: mode=FinalPartitioned, gby=[RegionID@0 as RegionID], aggr=[sum(hits.AdvEngineID), count(Int64(1)), avg(hits.ResolutionWidth), count(DISTINCT hits.UserID)] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([RegionID@0], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([RegionID@0], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[RegionID@0 as RegionID], aggr=[sum(hits.AdvEngineID), count(Int64(1)), avg(hits.ResolutionWidth), count(DISTINCT hits.UserID)] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[RegionID, UserID, ResolutionWidth, AdvEngineID], file_type=parquet └────────────────────────────────────────────────── "); @@ -231,21 +234,24 @@ mod tests { │ SortExec: TopK(fetch=10), expr=[u@1 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[MobilePhoneModel@0 as MobilePhoneModel, count(alias1)@1 as u] │ AggregateExec: mode=FinalPartitioned, gby=[MobilePhoneModel@0 as MobilePhoneModel], aggr=[count(alias1)] - │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([MobilePhoneModel@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[MobilePhoneModel@0 as MobilePhoneModel], aggr=[count(alias1)] - │ AggregateExec: mode=FinalPartitioned, gby=[MobilePhoneModel@0 as MobilePhoneModel, alias1@1 as alias1], aggr=[] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([MobilePhoneModel@0, alias1@1], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[MobilePhoneModel@1 as MobilePhoneModel, UserID@0 as alias1], aggr=[] - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: MobilePhoneModel@1 != - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] - │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[UserID, MobilePhoneModel], file_type=parquet, predicate=MobilePhoneModel@1 != , pruning_predicate=MobilePhoneModel_null_count@2 != row_count@3 AND (MobilePhoneModel_min@0 != OR != MobilePhoneModel_max@1), required_guarantees=[MobilePhoneModel not in ()] + │ RepartitionExec: partitioning=Hash([MobilePhoneModel@0], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[MobilePhoneModel@0 as MobilePhoneModel], aggr=[count(alias1)] + │ AggregateExec: mode=FinalPartitioned, gby=[MobilePhoneModel@0 as MobilePhoneModel, alias1@1 as alias1], aggr=[] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([MobilePhoneModel@0, alias1@1], 6), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[MobilePhoneModel@1 as MobilePhoneModel, UserID@0 as alias1], aggr=[] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: MobilePhoneModel@1 != + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[UserID, MobilePhoneModel], file_type=parquet, predicate=MobilePhoneModel@1 != , pruning_predicate=MobilePhoneModel_null_count@2 != row_count@3 AND (MobilePhoneModel_min@0 != OR != MobilePhoneModel_max@1), required_guarantees=[MobilePhoneModel not in ()] + └────────────────────────────────────────────────── "); Ok(()) } @@ -259,21 +265,24 @@ mod tests { │ SortExec: TopK(fetch=10), expr=[u@2 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel, count(alias1)@2 as u] │ AggregateExec: mode=FinalPartitioned, gby=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel], aggr=[count(alias1)] - │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([MobilePhone@0, MobilePhoneModel@1], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel], aggr=[count(alias1)] - │ AggregateExec: mode=FinalPartitioned, gby=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel, alias1@2 as alias1], aggr=[] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([MobilePhone@0, MobilePhoneModel@1, alias1@2], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[MobilePhone@1 as MobilePhone, MobilePhoneModel@2 as MobilePhoneModel, UserID@0 as alias1], aggr=[] - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: MobilePhoneModel@2 != - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] - │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[UserID, MobilePhone, MobilePhoneModel], file_type=parquet, predicate=MobilePhoneModel@2 != , pruning_predicate=MobilePhoneModel_null_count@2 != row_count@3 AND (MobilePhoneModel_min@0 != OR != MobilePhoneModel_max@1), required_guarantees=[MobilePhoneModel not in ()] + │ RepartitionExec: partitioning=Hash([MobilePhone@0, MobilePhoneModel@1], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel], aggr=[count(alias1)] + │ AggregateExec: mode=FinalPartitioned, gby=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel, alias1@2 as alias1], aggr=[] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([MobilePhone@0, MobilePhoneModel@1, alias1@2], 6), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[MobilePhone@1 as MobilePhone, MobilePhoneModel@2 as MobilePhoneModel, UserID@0 as alias1], aggr=[] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: MobilePhoneModel@2 != + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[UserID, MobilePhone, MobilePhoneModel], file_type=parquet, predicate=MobilePhoneModel@2 != , pruning_predicate=MobilePhoneModel_null_count@2 != row_count@3 AND (MobilePhoneModel_min@0 != OR != MobilePhoneModel_max@1), required_guarantees=[MobilePhoneModel not in ()] + └────────────────────────────────────────────────── "); Ok(()) } @@ -284,20 +293,23 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c@1 DESC], fetch=10 - │ SortExec: TopK(fetch=10), expr=[c@1 DESC], preserve_partitioning=[true] - │ ProjectionExec: expr=[SearchPhrase@0 as SearchPhrase, count(Int64(1))@1 as c] - │ AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([SearchPhrase@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(Int64(1))] - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: SearchPhrase@0 != - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] - │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[SearchPhrase], file_type=parquet, predicate=SearchPhrase@0 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + │ SortExec: TopK(fetch=10), expr=[c@1 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[SearchPhrase@0 as SearchPhrase, count(Int64(1))@1 as c] + │ AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(Int64(1))] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([SearchPhrase@0], 6), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(Int64(1))] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: SearchPhrase@0 != + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[SearchPhrase], file_type=parquet, predicate=SearchPhrase@0 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] + └────────────────────────────────────────────────── "); Ok(()) } @@ -311,21 +323,24 @@ mod tests { │ SortExec: TopK(fetch=10), expr=[u@1 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[SearchPhrase@0 as SearchPhrase, count(alias1)@1 as u] │ AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(alias1)] - │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([SearchPhrase@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(alias1)] - │ AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase, alias1@1 as alias1], aggr=[] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([SearchPhrase@0, alias1@1], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[SearchPhrase@1 as SearchPhrase, UserID@0 as alias1], aggr=[] - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: SearchPhrase@1 != - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] - │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[UserID, SearchPhrase], file_type=parquet, predicate=SearchPhrase@1 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] + │ RepartitionExec: partitioning=Hash([SearchPhrase@0], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(alias1)] + │ AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase, alias1@1 as alias1], aggr=[] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([SearchPhrase@0, alias1@1], 6), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[SearchPhrase@1 as SearchPhrase, UserID@0 as alias1], aggr=[] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: SearchPhrase@1 != + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[UserID, SearchPhrase], file_type=parquet, predicate=SearchPhrase@1 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] + └────────────────────────────────────────────────── "); Ok(()) } @@ -336,20 +351,23 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c@2 DESC], fetch=10 - │ SortExec: TopK(fetch=10), expr=[c@2 DESC], preserve_partitioning=[true] - │ ProjectionExec: expr=[SearchEngineID@0 as SearchEngineID, SearchPhrase@1 as SearchPhrase, count(Int64(1))@2 as c] - │ AggregateExec: mode=FinalPartitioned, gby=[SearchEngineID@0 as SearchEngineID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([SearchEngineID@0, SearchPhrase@1], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[SearchEngineID@0 as SearchEngineID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: SearchPhrase@1 != - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] - │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[SearchEngineID, SearchPhrase], file_type=parquet, predicate=SearchPhrase@1 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + │ SortExec: TopK(fetch=10), expr=[c@2 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[SearchEngineID@0 as SearchEngineID, SearchPhrase@1 as SearchPhrase, count(Int64(1))@2 as c] + │ AggregateExec: mode=FinalPartitioned, gby=[SearchEngineID@0 as SearchEngineID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([SearchEngineID@0, SearchPhrase@1], 6), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[SearchEngineID@0 as SearchEngineID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: SearchPhrase@1 != + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[SearchEngineID, SearchPhrase], file_type=parquet, predicate=SearchPhrase@1 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] + └────────────────────────────────────────────────── "); Ok(()) } @@ -367,13 +385,13 @@ mod tests { │ SortExec: TopK(fetch=10), expr=[count(*)@1 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[UserID@0 as UserID, count(Int64(1))@1 as count(*), count(Int64(1))@1 as count(Int64(1))] │ AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([UserID@0], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([UserID@0], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[UserID@0 as UserID], aggr=[count(Int64(1))] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[UserID], file_type=parquet └────────────────────────────────────────────────── "); @@ -393,13 +411,13 @@ mod tests { │ SortExec: TopK(fetch=10), expr=[count(*)@2 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase, count(Int64(1))@2 as count(*), count(Int64(1))@2 as count(Int64(1))] │ AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([UserID@0, SearchPhrase@1], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([UserID@0, SearchPhrase@1], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[UserID, SearchPhrase], file_type=parquet └────────────────────────────────────────────────── "); @@ -427,13 +445,13 @@ mod tests { │ SortExec: TopK(fetch=10), expr=[count(*)@3 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[UserID@0 as UserID, date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime))@1 as m, SearchPhrase@2 as SearchPhrase, count(Int64(1))@3 as count(*), count(Int64(1))@3 as count(Int64(1))] │ AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID, date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime))@1 as date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime)), SearchPhrase@2 as SearchPhrase], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([UserID@0, date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime))@1, SearchPhrase@2], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([UserID@0, date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime))@1, SearchPhrase@2], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[UserID@1 as UserID, date_part(MINUTE, to_timestamp_seconds(EventTime@0)) as date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime)), SearchPhrase@2 as SearchPhrase], aggr=[count(Int64(1))] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[EventTime, UserID, SearchPhrase], file_type=parquet └────────────────────────────────────────────────── "#); @@ -446,12 +464,12 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: UserID@0 = 435090932899640449 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[UserID], file_type=parquet, predicate=UserID@0 = 435090932899640449, pruning_predicate=UserID_null_count@2 != row_count@3 AND UserID_min@0 <= 435090932899640449 AND 435090932899640449 <= UserID_max@1, required_guarantees=[UserID in (435090932899640449)] └────────────────────────────────────────────────── "); @@ -466,14 +484,14 @@ mod tests { │ ProjectionExec: expr=[count(Int64(1))@0 as count(*)] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] │ ProjectionExec: expr=[] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: CAST(URL@0 AS Utf8View) LIKE %google% - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[URL], file_type=parquet, predicate=CAST(URL@0 AS Utf8View) LIKE %google% └────────────────────────────────────────────────── "); @@ -502,13 +520,13 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [EventTime@4 ASC NULLS LAST], fetch=10 - │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ SortExec: TopK(fetch=10), expr=[EventTime@4 ASC NULLS LAST], preserve_partitioning=[true] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: CAST(URL@13 AS Utf8View) LIKE %google% - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[WatchID, JavaEnable, Title, GoodEvent, EventTime, EventDate, CounterID, ClientIP, RegionID, UserID, CounterClass, OS, UserAgent, URL, Referer, IsRefresh, RefererCategoryID, RefererRegionID, URLCategoryID, URLRegionID, ResolutionWidth, ResolutionHeight, ResolutionDepth, FlashMajor, FlashMinor, FlashMinor2, NetMajor, NetMinor, UserAgentMajor, UserAgentMinor, CookieEnable, JavascriptEnable, IsMobile, MobilePhone, MobilePhoneModel, Params, IPNetworkID, TraficSourceID, SearchEngineID, SearchPhrase, AdvEngineID, IsArtifical, WindowClientWidth, WindowClientHeight, ClientTimeZone, ClientEventTime, SilverlightVersion1, SilverlightVersion2, SilverlightVersion3, SilverlightVersion4, PageCharset, CodeVersion, IsLink, IsDownload, IsNotBounce, FUniqID, OriginalURL, HID, IsOldCounter, IsEvent, IsParameter, DontCountHits, WithHash, HitColor, LocalEventTime, Age, Sex, Income, Interests, Robotness, RemoteIP, WindowName, OpenerName, HistoryLength, BrowserLanguage, BrowserCountry, SocialNetwork, SocialAction, HTTPError, SendTiming, DNSTiming, ConnectTiming, ResponseStartTiming, ResponseEndTiming, FetchTiming, SocialSourceNetworkID, SocialSourcePage, ParamPrice, ParamOrderID, ParamCurrency, ParamCurrencyID, OpenstatServiceName, OpenstatCampaignID, OpenstatAdID, OpenstatSourceID, UTMSource, UTMMedium, UTMCampaign, UTMContent, UTMTerm, FromTag, HasGCLID, RefererHash, URLHash, CLID], file_type=parquet, predicate=CAST(URL@13 AS Utf8View) LIKE %google% AND DynamicFilter [ empty ] └────────────────────────────────────────────────── "); @@ -529,13 +547,13 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [SearchPhrase@0 ASC NULLS LAST], fetch=10 - │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ SortExec: TopK(fetch=10), expr=[SearchPhrase@0 ASC NULLS LAST], preserve_partitioning=[true] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: SearchPhrase@0 != - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[SearchPhrase], file_type=parquet, predicate=SearchPhrase@0 != AND DynamicFilter [ empty ], pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] └────────────────────────────────────────────────── "); @@ -549,14 +567,14 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[SearchPhrase@0 as SearchPhrase] │ SortPreservingMergeExec: [EventTime@1 ASC NULLS LAST, SearchPhrase@0 ASC NULLS LAST], fetch=10 - │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ SortExec: TopK(fetch=10), expr=[EventTime@1 ASC NULLS LAST, SearchPhrase@0 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[SearchPhrase@1 as SearchPhrase, EventTime@0 as EventTime] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: SearchPhrase@1 != - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[EventTime, SearchPhrase], file_type=parquet, predicate=SearchPhrase@1 != AND DynamicFilter [ empty ], pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] └────────────────────────────────────────────────── "); @@ -569,22 +587,25 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [l@1 DESC], fetch=25 - │ SortExec: TopK(fetch=25), expr=[l@1 DESC], preserve_partitioning=[true] - │ ProjectionExec: expr=[CounterID@0 as CounterID, avg(length(hits.URL))@1 as l, count(Int64(1))@2 as c] - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: count(Int64(1))@2 > 100000 - │ AggregateExec: mode=FinalPartitioned, gby=[CounterID@0 as CounterID], aggr=[avg(length(hits.URL)), count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([CounterID@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[CounterID@0 as CounterID], aggr=[avg(length(hits.URL)), count(Int64(1))] - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: URL@1 != - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] - │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[CounterID, URL], file_type=parquet, predicate=URL@1 != , pruning_predicate=URL_null_count@2 != row_count@3 AND (URL_min@0 != OR != URL_max@1), required_guarantees=[URL not in ()] + │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + │ SortExec: TopK(fetch=25), expr=[l@1 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[CounterID@0 as CounterID, avg(length(hits.URL))@1 as l, count(Int64(1))@2 as c] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: count(Int64(1))@2 > 100000 + │ AggregateExec: mode=FinalPartitioned, gby=[CounterID@0 as CounterID], aggr=[avg(length(hits.URL)), count(Int64(1))] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([CounterID@0], 6), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[CounterID@0 as CounterID], aggr=[avg(length(hits.URL)), count(Int64(1))] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: URL@1 != + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[CounterID, URL], file_type=parquet, predicate=URL@1 != , pruning_predicate=URL_null_count@2 != row_count@3 AND (URL_min@0 != OR != URL_max@1), required_guarantees=[URL not in ()] + └────────────────────────────────────────────────── "); Ok(()) } @@ -595,22 +616,25 @@ mod tests { assert_snapshot!(display, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [l@1 DESC], fetch=25 - │ SortExec: TopK(fetch=25), expr=[l@1 DESC], preserve_partitioning=[true] - │ ProjectionExec: expr=[regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))@0 as k, avg(length(hits.Referer))@1 as l, count(Int64(1))@2 as c, min(hits.Referer)@3 as min(hits.Referer)] - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: count(Int64(1))@2 > 100000 - │ AggregateExec: mode=FinalPartitioned, gby=[regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))@0 as regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))], aggr=[avg(length(hits.Referer)), count(Int64(1)), min(hits.Referer)] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[regexp_replace(CAST(Referer@0 AS LargeUtf8), ^https?://(?:www\.)?([^/]+)/.*$, \1) as regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))], aggr=[avg(length(hits.Referer)), count(Int64(1)), min(hits.Referer)] - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: Referer@0 != - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] - │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[Referer], file_type=parquet, predicate=Referer@0 != , pruning_predicate=Referer_null_count@2 != row_count@3 AND (Referer_min@0 != OR != Referer_max@1), required_guarantees=[Referer not in ()] + │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + │ SortExec: TopK(fetch=25), expr=[l@1 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))@0 as k, avg(length(hits.Referer))@1 as l, count(Int64(1))@2 as c, min(hits.Referer)@3 as min(hits.Referer)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: count(Int64(1))@2 > 100000 + │ AggregateExec: mode=FinalPartitioned, gby=[regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))@0 as regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))], aggr=[avg(length(hits.Referer)), count(Int64(1)), min(hits.Referer)] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))@0], 6), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[regexp_replace(CAST(Referer@0 AS LargeUtf8), ^https?://(?:www\.)?([^/]+)/.*$, \1) as regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))], aggr=[avg(length(hits.Referer)), count(Int64(1)), min(hits.Referer)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: Referer@0 != + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[Referer], file_type=parquet, predicate=Referer@0 != , pruning_predicate=Referer_null_count@2 != row_count@3 AND (Referer_min@0 != OR != Referer_max@1), required_guarantees=[Referer not in ()] + └────────────────────────────────────────────────── "#); Ok(()) } @@ -622,12 +646,12 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ AggregateExec: mode=Final, gby=[], aggr=[sum(hits.ResolutionWidth), sum(hits.ResolutionWidth + Int64(1)), sum(hits.ResolutionWidth + Int64(2)), sum(hits.ResolutionWidth + Int64(3)), sum(hits.ResolutionWidth + Int64(4)), sum(hits.ResolutionWidth + Int64(5)), sum(hits.ResolutionWidth + Int64(6)), sum(hits.ResolutionWidth + Int64(7)), sum(hits.ResolutionWidth + Int64(8)), sum(hits.ResolutionWidth + Int64(9)), sum(hits.ResolutionWidth + Int64(10)), sum(hits.ResolutionWidth + Int64(11)), sum(hits.ResolutionWidth + Int64(12)), sum(hits.ResolutionWidth + Int64(13)), sum(hits.ResolutionWidth + Int64(14)), sum(hits.ResolutionWidth + Int64(15)), sum(hits.ResolutionWidth + Int64(16)), sum(hits.ResolutionWidth + Int64(17)), sum(hits.ResolutionWidth + Int64(18)), sum(hits.ResolutionWidth + Int64(19)), sum(hits.ResolutionWidth + Int64(20)), sum(hits.ResolutionWidth + Int64(21)), sum(hits.ResolutionWidth + Int64(22)), sum(hits.ResolutionWidth + Int64(23)), sum(hits.ResolutionWidth + Int64(24)), sum(hits.ResolutionWidth + Int64(25)), sum(hits.ResolutionWidth + Int64(26)), sum(hits.ResolutionWidth + Int64(27)), sum(hits.ResolutionWidth + Int64(28)), sum(hits.ResolutionWidth + Int64(29)), sum(hits.ResolutionWidth + Int64(30)), sum(hits.ResolutionWidth + Int64(31)), sum(hits.ResolutionWidth + Int64(32)), sum(hits.ResolutionWidth + Int64(33)), sum(hits.ResolutionWidth + Int64(34)), sum(hits.ResolutionWidth + Int64(35)), sum(hits.ResolutionWidth + Int64(36)), sum(hits.ResolutionWidth + Int64(37)), sum(hits.ResolutionWidth + Int64(38)), sum(hits.ResolutionWidth + Int64(39)), sum(hits.ResolutionWidth + Int64(40)), sum(hits.ResolutionWidth + Int64(41)), sum(hits.ResolutionWidth + Int64(42)), sum(hits.ResolutionWidth + Int64(43)), sum(hits.ResolutionWidth + Int64(44)), sum(hits.ResolutionWidth + Int64(45)), sum(hits.ResolutionWidth + Int64(46)), sum(hits.ResolutionWidth + Int64(47)), sum(hits.ResolutionWidth + Int64(48)), sum(hits.ResolutionWidth + Int64(49)), sum(hits.ResolutionWidth + Int64(50)), sum(hits.ResolutionWidth + Int64(51)), sum(hits.ResolutionWidth + Int64(52)), sum(hits.ResolutionWidth + Int64(53)), sum(hits.ResolutionWidth + Int64(54)), sum(hits.ResolutionWidth + Int64(55)), sum(hits.ResolutionWidth + Int64(56)), sum(hits.ResolutionWidth + Int64(57)), sum(hits.ResolutionWidth + Int64(58)), sum(hits.ResolutionWidth + Int64(59)), sum(hits.ResolutionWidth + Int64(60)), sum(hits.ResolutionWidth + Int64(61)), sum(hits.ResolutionWidth + Int64(62)), sum(hits.ResolutionWidth + Int64(63)), sum(hits.ResolutionWidth + Int64(64)), sum(hits.ResolutionWidth + Int64(65)), sum(hits.ResolutionWidth + Int64(66)), sum(hits.ResolutionWidth + Int64(67)), sum(hits.ResolutionWidth + Int64(68)), sum(hits.ResolutionWidth + Int64(69)), sum(hits.ResolutionWidth + Int64(70)), sum(hits.ResolutionWidth + Int64(71)), sum(hits.ResolutionWidth + Int64(72)), sum(hits.ResolutionWidth + Int64(73)), sum(hits.ResolutionWidth + Int64(74)), sum(hits.ResolutionWidth + Int64(75)), sum(hits.ResolutionWidth + Int64(76)), sum(hits.ResolutionWidth + Int64(77)), sum(hits.ResolutionWidth + Int64(78)), sum(hits.ResolutionWidth + Int64(79)), sum(hits.ResolutionWidth + Int64(80)), sum(hits.ResolutionWidth + Int64(81)), sum(hits.ResolutionWidth + Int64(82)), sum(hits.ResolutionWidth + Int64(83)), sum(hits.ResolutionWidth + Int64(84)), sum(hits.ResolutionWidth + Int64(85)), sum(hits.ResolutionWidth + Int64(86)), sum(hits.ResolutionWidth + Int64(87)), sum(hits.ResolutionWidth + Int64(88)), sum(hits.ResolutionWidth + Int64(89))] │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[sum(hits.ResolutionWidth), sum(hits.ResolutionWidth + Int64(1)), sum(hits.ResolutionWidth + Int64(2)), sum(hits.ResolutionWidth + Int64(3)), sum(hits.ResolutionWidth + Int64(4)), sum(hits.ResolutionWidth + Int64(5)), sum(hits.ResolutionWidth + Int64(6)), sum(hits.ResolutionWidth + Int64(7)), sum(hits.ResolutionWidth + Int64(8)), sum(hits.ResolutionWidth + Int64(9)), sum(hits.ResolutionWidth + Int64(10)), sum(hits.ResolutionWidth + Int64(11)), sum(hits.ResolutionWidth + Int64(12)), sum(hits.ResolutionWidth + Int64(13)), sum(hits.ResolutionWidth + Int64(14)), sum(hits.ResolutionWidth + Int64(15)), sum(hits.ResolutionWidth + Int64(16)), sum(hits.ResolutionWidth + Int64(17)), sum(hits.ResolutionWidth + Int64(18)), sum(hits.ResolutionWidth + Int64(19)), sum(hits.ResolutionWidth + Int64(20)), sum(hits.ResolutionWidth + Int64(21)), sum(hits.ResolutionWidth + Int64(22)), sum(hits.ResolutionWidth + Int64(23)), sum(hits.ResolutionWidth + Int64(24)), sum(hits.ResolutionWidth + Int64(25)), sum(hits.ResolutionWidth + Int64(26)), sum(hits.ResolutionWidth + Int64(27)), sum(hits.ResolutionWidth + Int64(28)), sum(hits.ResolutionWidth + Int64(29)), sum(hits.ResolutionWidth + Int64(30)), sum(hits.ResolutionWidth + Int64(31)), sum(hits.ResolutionWidth + Int64(32)), sum(hits.ResolutionWidth + Int64(33)), sum(hits.ResolutionWidth + Int64(34)), sum(hits.ResolutionWidth + Int64(35)), sum(hits.ResolutionWidth + Int64(36)), sum(hits.ResolutionWidth + Int64(37)), sum(hits.ResolutionWidth + Int64(38)), sum(hits.ResolutionWidth + Int64(39)), sum(hits.ResolutionWidth + Int64(40)), sum(hits.ResolutionWidth + Int64(41)), sum(hits.ResolutionWidth + Int64(42)), sum(hits.ResolutionWidth + Int64(43)), sum(hits.ResolutionWidth + Int64(44)), sum(hits.ResolutionWidth + Int64(45)), sum(hits.ResolutionWidth + Int64(46)), sum(hits.ResolutionWidth + Int64(47)), sum(hits.ResolutionWidth + Int64(48)), sum(hits.ResolutionWidth + Int64(49)), sum(hits.ResolutionWidth + Int64(50)), sum(hits.ResolutionWidth + Int64(51)), sum(hits.ResolutionWidth + Int64(52)), sum(hits.ResolutionWidth + Int64(53)), sum(hits.ResolutionWidth + Int64(54)), sum(hits.ResolutionWidth + Int64(55)), sum(hits.ResolutionWidth + Int64(56)), sum(hits.ResolutionWidth + Int64(57)), sum(hits.ResolutionWidth + Int64(58)), sum(hits.ResolutionWidth + Int64(59)), sum(hits.ResolutionWidth + Int64(60)), sum(hits.ResolutionWidth + Int64(61)), sum(hits.ResolutionWidth + Int64(62)), sum(hits.ResolutionWidth + Int64(63)), sum(hits.ResolutionWidth + Int64(64)), sum(hits.ResolutionWidth + Int64(65)), sum(hits.ResolutionWidth + Int64(66)), sum(hits.ResolutionWidth + Int64(67)), sum(hits.ResolutionWidth + Int64(68)), sum(hits.ResolutionWidth + Int64(69)), sum(hits.ResolutionWidth + Int64(70)), sum(hits.ResolutionWidth + Int64(71)), sum(hits.ResolutionWidth + Int64(72)), sum(hits.ResolutionWidth + Int64(73)), sum(hits.ResolutionWidth + Int64(74)), sum(hits.ResolutionWidth + Int64(75)), sum(hits.ResolutionWidth + Int64(76)), sum(hits.ResolutionWidth + Int64(77)), sum(hits.ResolutionWidth + Int64(78)), sum(hits.ResolutionWidth + Int64(79)), sum(hits.ResolutionWidth + Int64(80)), sum(hits.ResolutionWidth + Int64(81)), sum(hits.ResolutionWidth + Int64(82)), sum(hits.ResolutionWidth + Int64(83)), sum(hits.ResolutionWidth + Int64(84)), sum(hits.ResolutionWidth + Int64(85)), sum(hits.ResolutionWidth + Int64(86)), sum(hits.ResolutionWidth + Int64(87)), sum(hits.ResolutionWidth + Int64(88)), sum(hits.ResolutionWidth + Int64(89))] │ ProjectionExec: expr=[CAST(ResolutionWidth@0 AS Int64) as __common_expr_1] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[ResolutionWidth], file_type=parquet └────────────────────────────────────────────────── "); @@ -640,20 +664,23 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c@2 DESC], fetch=10 - │ SortExec: TopK(fetch=10), expr=[c@2 DESC], preserve_partitioning=[true] - │ ProjectionExec: expr=[SearchEngineID@0 as SearchEngineID, ClientIP@1 as ClientIP, count(Int64(1))@2 as c, sum(hits.IsRefresh)@3 as sum(hits.IsRefresh), avg(hits.ResolutionWidth)@4 as avg(hits.ResolutionWidth)] - │ AggregateExec: mode=FinalPartitioned, gby=[SearchEngineID@0 as SearchEngineID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([SearchEngineID@0, ClientIP@1], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[SearchEngineID@3 as SearchEngineID, ClientIP@0 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)] - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: SearchPhrase@4 != , projection=[ClientIP@0, IsRefresh@1, ResolutionWidth@2, SearchEngineID@3] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] - │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[ClientIP, IsRefresh, ResolutionWidth, SearchEngineID, SearchPhrase], file_type=parquet, predicate=SearchPhrase@4 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + │ SortExec: TopK(fetch=10), expr=[c@2 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[SearchEngineID@0 as SearchEngineID, ClientIP@1 as ClientIP, count(Int64(1))@2 as c, sum(hits.IsRefresh)@3 as sum(hits.IsRefresh), avg(hits.ResolutionWidth)@4 as avg(hits.ResolutionWidth)] + │ AggregateExec: mode=FinalPartitioned, gby=[SearchEngineID@0 as SearchEngineID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([SearchEngineID@0, ClientIP@1], 6), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[SearchEngineID@3 as SearchEngineID, ClientIP@0 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: SearchPhrase@4 != , projection=[ClientIP@0, IsRefresh@1, ResolutionWidth@2, SearchEngineID@3] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[ClientIP, IsRefresh, ResolutionWidth, SearchEngineID, SearchPhrase], file_type=parquet, predicate=SearchPhrase@4 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] + └────────────────────────────────────────────────── "); Ok(()) } @@ -686,13 +713,13 @@ mod tests { │ SortExec: TopK(fetch=10), expr=[c@1 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[URL@0 as URL, count(Int64(1))@1 as c] │ AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([URL@0], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([URL@0], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[URL], file_type=parquet └────────────────────────────────────────────────── "); @@ -711,13 +738,13 @@ mod tests { │ SortExec: TopK(fetch=10), expr=[c@2 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[1 as Int64(1), URL@0 as URL, count(Int64(1))@1 as c] │ AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([URL@0], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([URL@0], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[URL], file_type=parquet └────────────────────────────────────────────────── "); @@ -736,14 +763,14 @@ mod tests { │ SortExec: TopK(fetch=10), expr=[c@4 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[ClientIP@0 as ClientIP, hits.ClientIP - Int64(1)@1 as hits.ClientIP - Int64(1), hits.ClientIP - Int64(2)@2 as hits.ClientIP - Int64(2), hits.ClientIP - Int64(3)@3 as hits.ClientIP - Int64(3), count(Int64(1))@4 as c] │ AggregateExec: mode=FinalPartitioned, gby=[ClientIP@0 as ClientIP, hits.ClientIP - Int64(1)@1 as hits.ClientIP - Int64(1), hits.ClientIP - Int64(2)@2 as hits.ClientIP - Int64(2), hits.ClientIP - Int64(3)@3 as hits.ClientIP - Int64(3)], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ClientIP@0, hits.ClientIP - Int64(1)@1, hits.ClientIP - Int64(2)@2, hits.ClientIP - Int64(3)@3], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([ClientIP@0, hits.ClientIP - Int64(1)@1, hits.ClientIP - Int64(2)@2, hits.ClientIP - Int64(3)@3], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[ClientIP@1 as ClientIP, __common_expr_1@0 - 1 as hits.ClientIP - Int64(1), __common_expr_1@0 - 2 as hits.ClientIP - Int64(2), __common_expr_1@0 - 3 as hits.ClientIP - Int64(3)], aggr=[count(Int64(1))] │ ProjectionExec: expr=[CAST(ClientIP@0 AS Int64) as __common_expr_1, ClientIP@0 as ClientIP] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[ClientIP], file_type=parquet └────────────────────────────────────────────────── "); @@ -756,20 +783,23 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [pageviews@1 DESC], fetch=10 - │ SortExec: TopK(fetch=10), expr=[pageviews@1 DESC], preserve_partitioning=[true] - │ ProjectionExec: expr=[URL@0 as URL, count(Int64(1))@1 as pageviews] - │ AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([URL@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))] - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: CounterID@1 = 62 AND CAST(EventDate@0 AS Utf8) >= 2013-07-01 AND CAST(EventDate@0 AS Utf8) <= 2013-07-31 AND DontCountHits@4 = 0 AND IsRefresh@3 = 0 AND URL@2 != , projection=[URL@2] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] - │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[EventDate, CounterID, URL, IsRefresh, DontCountHits], file_type=parquet, predicate=CounterID@1 = 62 AND CAST(EventDate@0 AS Utf8) >= 2013-07-01 AND CAST(EventDate@0 AS Utf8) <= 2013-07-31 AND DontCountHits@4 = 0 AND IsRefresh@3 = 0 AND URL@2 != , pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND DontCountHits_null_count@6 != row_count@3 AND DontCountHits_min@4 <= 0 AND 0 <= DontCountHits_max@5 AND IsRefresh_null_count@9 != row_count@3 AND IsRefresh_min@7 <= 0 AND 0 <= IsRefresh_max@8 AND URL_null_count@12 != row_count@3 AND (URL_min@10 != OR != URL_max@11), required_guarantees=[CounterID in (62), DontCountHits in (0), IsRefresh in (0), URL not in ()] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + │ SortExec: TopK(fetch=10), expr=[pageviews@1 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[URL@0 as URL, count(Int64(1))@1 as pageviews] + │ AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([URL@0], 6), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: CounterID@1 = 62 AND CAST(EventDate@0 AS Utf8) >= 2013-07-01 AND CAST(EventDate@0 AS Utf8) <= 2013-07-31 AND DontCountHits@4 = 0 AND IsRefresh@3 = 0 AND URL@2 != , projection=[URL@2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[EventDate, CounterID, URL, IsRefresh, DontCountHits], file_type=parquet, predicate=CounterID@1 = 62 AND CAST(EventDate@0 AS Utf8) >= 2013-07-01 AND CAST(EventDate@0 AS Utf8) <= 2013-07-31 AND DontCountHits@4 = 0 AND IsRefresh@3 = 0 AND URL@2 != , pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND DontCountHits_null_count@6 != row_count@3 AND DontCountHits_min@4 <= 0 AND 0 <= DontCountHits_max@5 AND IsRefresh_null_count@9 != row_count@3 AND IsRefresh_min@7 <= 0 AND 0 <= IsRefresh_max@8 AND URL_null_count@12 != row_count@3 AND (URL_min@10 != OR != URL_max@11), required_guarantees=[CounterID in (62), DontCountHits in (0), IsRefresh in (0), URL not in ()] + └────────────────────────────────────────────────── "); Ok(()) } @@ -780,20 +810,23 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [pageviews@1 DESC], fetch=10 - │ SortExec: TopK(fetch=10), expr=[pageviews@1 DESC], preserve_partitioning=[true] - │ ProjectionExec: expr=[Title@0 as Title, count(Int64(1))@1 as pageviews] - │ AggregateExec: mode=FinalPartitioned, gby=[Title@0 as Title], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([Title@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[Title@0 as Title], aggr=[count(Int64(1))] - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: CounterID@2 = 62 AND CAST(EventDate@1 AS Utf8) >= 2013-07-01 AND CAST(EventDate@1 AS Utf8) <= 2013-07-31 AND DontCountHits@4 = 0 AND IsRefresh@3 = 0 AND Title@0 != , projection=[Title@0] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] - │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[Title, EventDate, CounterID, IsRefresh, DontCountHits], file_type=parquet, predicate=CounterID@2 = 62 AND CAST(EventDate@1 AS Utf8) >= 2013-07-01 AND CAST(EventDate@1 AS Utf8) <= 2013-07-31 AND DontCountHits@4 = 0 AND IsRefresh@3 = 0 AND Title@0 != , pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND DontCountHits_null_count@6 != row_count@3 AND DontCountHits_min@4 <= 0 AND 0 <= DontCountHits_max@5 AND IsRefresh_null_count@9 != row_count@3 AND IsRefresh_min@7 <= 0 AND 0 <= IsRefresh_max@8 AND Title_null_count@12 != row_count@3 AND (Title_min@10 != OR != Title_max@11), required_guarantees=[CounterID in (62), DontCountHits in (0), IsRefresh in (0), Title not in ()] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + │ SortExec: TopK(fetch=10), expr=[pageviews@1 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[Title@0 as Title, count(Int64(1))@1 as pageviews] + │ AggregateExec: mode=FinalPartitioned, gby=[Title@0 as Title], aggr=[count(Int64(1))] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([Title@0], 6), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[Title@0 as Title], aggr=[count(Int64(1))] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: CounterID@2 = 62 AND CAST(EventDate@1 AS Utf8) >= 2013-07-01 AND CAST(EventDate@1 AS Utf8) <= 2013-07-31 AND DontCountHits@4 = 0 AND IsRefresh@3 = 0 AND Title@0 != , projection=[Title@0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[Title, EventDate, CounterID, IsRefresh, DontCountHits], file_type=parquet, predicate=CounterID@2 = 62 AND CAST(EventDate@1 AS Utf8) >= 2013-07-01 AND CAST(EventDate@1 AS Utf8) <= 2013-07-31 AND DontCountHits@4 = 0 AND IsRefresh@3 = 0 AND Title@0 != , pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND DontCountHits_null_count@6 != row_count@3 AND DontCountHits_min@4 <= 0 AND 0 <= DontCountHits_max@5 AND IsRefresh_null_count@9 != row_count@3 AND IsRefresh_min@7 <= 0 AND 0 <= IsRefresh_max@8 AND Title_null_count@12 != row_count@3 AND (Title_min@10 != OR != Title_max@11), required_guarantees=[CounterID in (62), DontCountHits in (0), IsRefresh in (0), Title not in ()] + └────────────────────────────────────────────────── "); Ok(()) } @@ -805,20 +838,23 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ GlobalLimitExec: skip=1000, fetch=10 │ SortPreservingMergeExec: [pageviews@1 DESC], fetch=1010 - │ SortExec: TopK(fetch=1010), expr=[pageviews@1 DESC], preserve_partitioning=[true] - │ ProjectionExec: expr=[URL@0 as URL, count(Int64(1))@1 as pageviews] - │ AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([URL@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))] - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: CounterID@1 = 62 AND CAST(EventDate@0 AS Utf8) >= 2013-07-01 AND CAST(EventDate@0 AS Utf8) <= 2013-07-31 AND IsRefresh@3 = 0 AND IsLink@4 != 0 AND IsDownload@5 = 0, projection=[URL@2] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] - │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[EventDate, CounterID, URL, IsRefresh, IsLink, IsDownload], file_type=parquet, predicate=CounterID@1 = 62 AND CAST(EventDate@0 AS Utf8) >= 2013-07-01 AND CAST(EventDate@0 AS Utf8) <= 2013-07-31 AND IsRefresh@3 = 0 AND IsLink@4 != 0 AND IsDownload@5 = 0, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND IsRefresh_null_count@6 != row_count@3 AND IsRefresh_min@4 <= 0 AND 0 <= IsRefresh_max@5 AND IsLink_null_count@9 != row_count@3 AND (IsLink_min@7 != 0 OR 0 != IsLink_max@8) AND IsDownload_null_count@12 != row_count@3 AND IsDownload_min@10 <= 0 AND 0 <= IsDownload_max@11, required_guarantees=[CounterID in (62), IsDownload in (0), IsLink not in (0), IsRefresh in (0)] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + │ SortExec: TopK(fetch=1010), expr=[pageviews@1 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[URL@0 as URL, count(Int64(1))@1 as pageviews] + │ AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([URL@0], 6), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: CounterID@1 = 62 AND CAST(EventDate@0 AS Utf8) >= 2013-07-01 AND CAST(EventDate@0 AS Utf8) <= 2013-07-31 AND IsRefresh@3 = 0 AND IsLink@4 != 0 AND IsDownload@5 = 0, projection=[URL@2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[EventDate, CounterID, URL, IsRefresh, IsLink, IsDownload], file_type=parquet, predicate=CounterID@1 = 62 AND CAST(EventDate@0 AS Utf8) >= 2013-07-01 AND CAST(EventDate@0 AS Utf8) <= 2013-07-31 AND IsRefresh@3 = 0 AND IsLink@4 != 0 AND IsDownload@5 = 0, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND IsRefresh_null_count@6 != row_count@3 AND IsRefresh_min@4 <= 0 AND 0 <= IsRefresh_max@5 AND IsLink_null_count@9 != row_count@3 AND (IsLink_min@7 != 0 OR 0 != IsLink_max@8) AND IsDownload_null_count@12 != row_count@3 AND IsDownload_min@10 <= 0 AND 0 <= IsDownload_max@11, required_guarantees=[CounterID in (62), IsDownload in (0), IsLink not in (0), IsRefresh in (0)] + └────────────────────────────────────────────────── "); Ok(()) } @@ -830,20 +866,23 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ GlobalLimitExec: skip=1000, fetch=10 │ SortPreservingMergeExec: [pageviews@5 DESC], fetch=1010 - │ SortExec: TopK(fetch=1010), expr=[pageviews@5 DESC], preserve_partitioning=[true] - │ ProjectionExec: expr=[TraficSourceID@0 as TraficSourceID, SearchEngineID@1 as SearchEngineID, AdvEngineID@2 as AdvEngineID, CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END@3 as src, URL@4 as dst, count(Int64(1))@5 as pageviews] - │ AggregateExec: mode=FinalPartitioned, gby=[TraficSourceID@0 as TraficSourceID, SearchEngineID@1 as SearchEngineID, AdvEngineID@2 as AdvEngineID, CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END@3 as CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END, URL@4 as URL], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([TraficSourceID@0, SearchEngineID@1, AdvEngineID@2, CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END@3, URL@4], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[TraficSourceID@2 as TraficSourceID, SearchEngineID@3 as SearchEngineID, AdvEngineID@4 as AdvEngineID, CASE WHEN SearchEngineID@3 = 0 AND AdvEngineID@4 = 0 THEN Referer@1 ELSE END as CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END, URL@0 as URL], aggr=[count(Int64(1))] - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: CounterID@1 = 62 AND CAST(EventDate@0 AS Utf8) >= 2013-07-01 AND CAST(EventDate@0 AS Utf8) <= 2013-07-31 AND IsRefresh@4 = 0, projection=[URL@2, Referer@3, TraficSourceID@5, SearchEngineID@6, AdvEngineID@7] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] - │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[EventDate, CounterID, URL, Referer, IsRefresh, TraficSourceID, SearchEngineID, AdvEngineID], file_type=parquet, predicate=CounterID@1 = 62 AND CAST(EventDate@0 AS Utf8) >= 2013-07-01 AND CAST(EventDate@0 AS Utf8) <= 2013-07-31 AND IsRefresh@4 = 0, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND IsRefresh_null_count@6 != row_count@3 AND IsRefresh_min@4 <= 0 AND 0 <= IsRefresh_max@5, required_guarantees=[CounterID in (62), IsRefresh in (0)] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + │ SortExec: TopK(fetch=1010), expr=[pageviews@5 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[TraficSourceID@0 as TraficSourceID, SearchEngineID@1 as SearchEngineID, AdvEngineID@2 as AdvEngineID, CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END@3 as src, URL@4 as dst, count(Int64(1))@5 as pageviews] + │ AggregateExec: mode=FinalPartitioned, gby=[TraficSourceID@0 as TraficSourceID, SearchEngineID@1 as SearchEngineID, AdvEngineID@2 as AdvEngineID, CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END@3 as CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END, URL@4 as URL], aggr=[count(Int64(1))] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([TraficSourceID@0, SearchEngineID@1, AdvEngineID@2, CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END@3, URL@4], 6), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[TraficSourceID@2 as TraficSourceID, SearchEngineID@3 as SearchEngineID, AdvEngineID@4 as AdvEngineID, CASE WHEN SearchEngineID@3 = 0 AND AdvEngineID@4 = 0 THEN Referer@1 ELSE END as CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END, URL@0 as URL], aggr=[count(Int64(1))] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: CounterID@1 = 62 AND CAST(EventDate@0 AS Utf8) >= 2013-07-01 AND CAST(EventDate@0 AS Utf8) <= 2013-07-31 AND IsRefresh@4 = 0, projection=[URL@2, Referer@3, TraficSourceID@5, SearchEngineID@6, AdvEngineID@7] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[EventDate, CounterID, URL, Referer, IsRefresh, TraficSourceID, SearchEngineID, AdvEngineID], file_type=parquet, predicate=CounterID@1 = 62 AND CAST(EventDate@0 AS Utf8) >= 2013-07-01 AND CAST(EventDate@0 AS Utf8) <= 2013-07-31 AND IsRefresh@4 = 0, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND IsRefresh_null_count@6 != row_count@3 AND IsRefresh_min@4 <= 0 AND 0 <= IsRefresh_max@5, required_guarantees=[CounterID in (62), IsRefresh in (0)] + └────────────────────────────────────────────────── "#); Ok(()) } @@ -855,20 +894,23 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ GlobalLimitExec: skip=100, fetch=10 │ SortPreservingMergeExec: [pageviews@2 DESC], fetch=110 - │ SortExec: TopK(fetch=110), expr=[pageviews@2 DESC], preserve_partitioning=[true] - │ ProjectionExec: expr=[URLHash@0 as URLHash, EventDate@1 as EventDate, count(Int64(1))@2 as pageviews] - │ AggregateExec: mode=FinalPartitioned, gby=[URLHash@0 as URLHash, EventDate@1 as EventDate], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([URLHash@0, EventDate@1], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[URLHash@1 as URLHash, EventDate@0 as EventDate], aggr=[count(Int64(1))] - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: CounterID@1 = 62 AND CAST(EventDate@0 AS Utf8) >= 2013-07-01 AND CAST(EventDate@0 AS Utf8) <= 2013-07-31 AND IsRefresh@2 = 0 AND (TraficSourceID@3 = -1 OR TraficSourceID@3 = 6) AND RefererHash@4 = 3594120000172545465, projection=[EventDate@0, URLHash@5] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] - │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[EventDate, CounterID, IsRefresh, TraficSourceID, RefererHash, URLHash], file_type=parquet, predicate=CounterID@1 = 62 AND CAST(EventDate@0 AS Utf8) >= 2013-07-01 AND CAST(EventDate@0 AS Utf8) <= 2013-07-31 AND IsRefresh@2 = 0 AND (TraficSourceID@3 = -1 OR TraficSourceID@3 = 6) AND RefererHash@4 = 3594120000172545465, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND IsRefresh_null_count@6 != row_count@3 AND IsRefresh_min@4 <= 0 AND 0 <= IsRefresh_max@5 AND (TraficSourceID_null_count@9 != row_count@3 AND TraficSourceID_min@7 <= -1 AND -1 <= TraficSourceID_max@8 OR TraficSourceID_null_count@9 != row_count@3 AND TraficSourceID_min@7 <= 6 AND 6 <= TraficSourceID_max@8) AND RefererHash_null_count@12 != row_count@3 AND RefererHash_min@10 <= 3594120000172545465 AND 3594120000172545465 <= RefererHash_max@11, required_guarantees=[CounterID in (62), IsRefresh in (0), RefererHash in (3594120000172545465), TraficSourceID in (-1, 6)] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + │ SortExec: TopK(fetch=110), expr=[pageviews@2 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[URLHash@0 as URLHash, EventDate@1 as EventDate, count(Int64(1))@2 as pageviews] + │ AggregateExec: mode=FinalPartitioned, gby=[URLHash@0 as URLHash, EventDate@1 as EventDate], aggr=[count(Int64(1))] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([URLHash@0, EventDate@1], 6), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[URLHash@1 as URLHash, EventDate@0 as EventDate], aggr=[count(Int64(1))] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: CounterID@1 = 62 AND CAST(EventDate@0 AS Utf8) >= 2013-07-01 AND CAST(EventDate@0 AS Utf8) <= 2013-07-31 AND IsRefresh@2 = 0 AND (TraficSourceID@3 = -1 OR TraficSourceID@3 = 6) AND RefererHash@4 = 3594120000172545465, projection=[EventDate@0, URLHash@5] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[EventDate, CounterID, IsRefresh, TraficSourceID, RefererHash, URLHash], file_type=parquet, predicate=CounterID@1 = 62 AND CAST(EventDate@0 AS Utf8) >= 2013-07-01 AND CAST(EventDate@0 AS Utf8) <= 2013-07-31 AND IsRefresh@2 = 0 AND (TraficSourceID@3 = -1 OR TraficSourceID@3 = 6) AND RefererHash@4 = 3594120000172545465, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND IsRefresh_null_count@6 != row_count@3 AND IsRefresh_min@4 <= 0 AND 0 <= IsRefresh_max@5 AND (TraficSourceID_null_count@9 != row_count@3 AND TraficSourceID_min@7 <= -1 AND -1 <= TraficSourceID_max@8 OR TraficSourceID_null_count@9 != row_count@3 AND TraficSourceID_min@7 <= 6 AND 6 <= TraficSourceID_max@8) AND RefererHash_null_count@12 != row_count@3 AND RefererHash_min@10 <= 3594120000172545465 AND 3594120000172545465 <= RefererHash_max@11, required_guarantees=[CounterID in (62), IsRefresh in (0), RefererHash in (3594120000172545465), TraficSourceID in (-1, 6)] + └────────────────────────────────────────────────── "); Ok(()) } @@ -880,20 +922,23 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ GlobalLimitExec: skip=10000, fetch=10 │ SortPreservingMergeExec: [pageviews@2 DESC], fetch=10010 - │ SortExec: TopK(fetch=10010), expr=[pageviews@2 DESC], preserve_partitioning=[true] - │ ProjectionExec: expr=[WindowClientWidth@0 as WindowClientWidth, WindowClientHeight@1 as WindowClientHeight, count(Int64(1))@2 as pageviews] - │ AggregateExec: mode=FinalPartitioned, gby=[WindowClientWidth@0 as WindowClientWidth, WindowClientHeight@1 as WindowClientHeight], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([WindowClientWidth@0, WindowClientHeight@1], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[WindowClientWidth@0 as WindowClientWidth, WindowClientHeight@1 as WindowClientHeight], aggr=[count(Int64(1))] - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: CounterID@1 = 62 AND CAST(EventDate@0 AS Utf8) >= 2013-07-01 AND CAST(EventDate@0 AS Utf8) <= 2013-07-31 AND IsRefresh@2 = 0 AND DontCountHits@5 = 0 AND URLHash@6 = 2868770270353813622, projection=[WindowClientWidth@3, WindowClientHeight@4] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__] t1:[__,__,__,p0,p1] - │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[EventDate, CounterID, IsRefresh, WindowClientWidth, WindowClientHeight, DontCountHits, URLHash], file_type=parquet, predicate=CounterID@1 = 62 AND CAST(EventDate@0 AS Utf8) >= 2013-07-01 AND CAST(EventDate@0 AS Utf8) <= 2013-07-31 AND IsRefresh@2 = 0 AND DontCountHits@5 = 0 AND URLHash@6 = 2868770270353813622, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND IsRefresh_null_count@6 != row_count@3 AND IsRefresh_min@4 <= 0 AND 0 <= IsRefresh_max@5 AND DontCountHits_null_count@9 != row_count@3 AND DontCountHits_min@7 <= 0 AND 0 <= DontCountHits_max@8 AND URLHash_null_count@12 != row_count@3 AND URLHash_min@10 <= 2868770270353813622 AND 2868770270353813622 <= URLHash_max@11, required_guarantees=[CounterID in (62), DontCountHits in (0), IsRefresh in (0), URLHash in (2868770270353813622)] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + │ SortExec: TopK(fetch=10010), expr=[pageviews@2 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[WindowClientWidth@0 as WindowClientWidth, WindowClientHeight@1 as WindowClientHeight, count(Int64(1))@2 as pageviews] + │ AggregateExec: mode=FinalPartitioned, gby=[WindowClientWidth@0 as WindowClientWidth, WindowClientHeight@1 as WindowClientHeight], aggr=[count(Int64(1))] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([WindowClientWidth@0, WindowClientHeight@1], 6), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[WindowClientWidth@0 as WindowClientWidth, WindowClientHeight@1 as WindowClientHeight], aggr=[count(Int64(1))] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: CounterID@1 = 62 AND CAST(EventDate@0 AS Utf8) >= 2013-07-01 AND CAST(EventDate@0 AS Utf8) <= 2013-07-31 AND IsRefresh@2 = 0 AND DontCountHits@5 = 0 AND URLHash@6 = 2868770270353813622, projection=[WindowClientWidth@3, WindowClientHeight@4] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__] t1:[__,__,p0,p1,__] t2:[__,__,__,__,p0] + │ DataSourceExec: file_groups={5 groups: [[/testdata/clickbench/plans_range0-3/hits/0.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/1.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..], [/testdata/clickbench/plans_range0-3/hits/2.parquet:..]]}, projection=[EventDate, CounterID, IsRefresh, WindowClientWidth, WindowClientHeight, DontCountHits, URLHash], file_type=parquet, predicate=CounterID@1 = 62 AND CAST(EventDate@0 AS Utf8) >= 2013-07-01 AND CAST(EventDate@0 AS Utf8) <= 2013-07-31 AND IsRefresh@2 = 0 AND DontCountHits@5 = 0 AND URLHash@6 = 2868770270353813622, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND IsRefresh_null_count@6 != row_count@3 AND IsRefresh_min@4 <= 0 AND 0 <= IsRefresh_max@5 AND DontCountHits_null_count@9 != row_count@3 AND DontCountHits_min@7 <= 0 AND 0 <= DontCountHits_max@8 AND URLHash_null_count@12 != row_count@3 AND URLHash_min@10 <= 2868770270353813622 AND 2868770270353813622 <= URLHash_max@11, required_guarantees=[CounterID in (62), DontCountHits in (0), IsRefresh in (0), URLHash in (2868770270353813622)] + └────────────────────────────────────────────────── "); Ok(()) } diff --git a/tests/tpcds_plans_test.rs b/tests/tpcds_plans_test.rs index d117bda7..5c80adac 100644 --- a/tests/tpcds_plans_test.rs +++ b/tests/tpcds_plans_test.rs @@ -260,7 +260,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, ws_bill_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, ws_sold_date_sk@9, ws_ext_discount_amt@11, ws_ext_sales_price@12, ws_ext_wholesale_cost@13, ws_ext_list_price@14] │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)], filter=CASE WHEN year_total@2 > Some(0),24,6 THEN year_total@3 / year_total@2 END > CASE WHEN year_total@0 > Some(0),24,6 THEN year_total@1 / year_total@0 END, projection=[customer_id@0, customer_id@2, customer_first_name@3, customer_last_name@4, customer_preferred_cust_flag@5, year_total@7, year_total@9] │ CoalescePartitionsExec @@ -284,7 +284,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, cs_bill_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, cs_sold_date_sk@9, cs_ext_discount_amt@11, cs_ext_sales_price@12, cs_ext_wholesale_cost@13, cs_ext_list_price@14] │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)] │ CoalescePartitionsExec @@ -304,7 +304,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, ss_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, ss_sold_date_sk@9, ss_ext_discount_amt@11, ss_ext_sales_price@12, ss_ext_wholesale_cost@13, ss_ext_list_price@14] │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[c_customer_id@0 as customer_id, c_first_name@1 as customer_first_name, c_last_name@2 as customer_last_name, c_preferred_cust_flag@3 as customer_preferred_cust_flag, sum(store_sales.ss_ext_list_price - store_sales.ss_ext_wholesale_cost - store_sales.ss_ext_discount_amt + store_sales.ss_ext_sales_price / Int64(2))@8 as year_total] │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@7 as d_year], aggr=[sum(store_sales.ss_ext_list_price - store_sales.ss_ext_wholesale_cost - store_sales.ss_ext_discount_amt + store_sales.ss_ext_sales_price / Int64(2))], ordering_mode=PartiallySorted([7]) │ CoalesceBatchesExec: target_batch_size=8192 @@ -318,7 +318,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, ss_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, ss_sold_date_sk@9, ss_ext_discount_amt@11, ss_ext_sales_price@12, ss_ext_wholesale_cost@13, ss_ext_list_price@14] │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(catalog_sales.cs_ext_list_price - catalog_sales.cs_ext_wholesale_cost - catalog_sales.cs_ext_discount_amt + catalog_sales.cs_ext_sales_price / Int64(2))@8 as year_total] │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@7 as d_year], aggr=[sum(catalog_sales.cs_ext_list_price - catalog_sales.cs_ext_wholesale_cost - catalog_sales.cs_ext_discount_amt + catalog_sales.cs_ext_sales_price / Int64(2))], ordering_mode=PartiallySorted([7]) │ CoalesceBatchesExec: target_batch_size=8192 @@ -332,7 +332,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, cs_bill_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, cs_sold_date_sk@9, cs_ext_discount_amt@11, cs_ext_sales_price@12, cs_ext_wholesale_cost@13, cs_ext_list_price@14] │ [Stage 14] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(web_sales.ws_ext_list_price - web_sales.ws_ext_wholesale_cost - web_sales.ws_ext_discount_amt + web_sales.ws_ext_sales_price / Int64(2))@8 as year_total] │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@7 as d_year], aggr=[sum(web_sales.ws_ext_list_price - web_sales.ws_ext_wholesale_cost - web_sales.ws_ext_discount_amt + web_sales.ws_ext_sales_price / Int64(2))], ordering_mode=PartiallySorted([7]) │ CoalesceBatchesExec: target_batch_size=8192 @@ -346,7 +346,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, ws_bill_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, ws_sold_date_sk@9, ws_ext_discount_amt@11, ws_ext_sales_price@12, ws_ext_wholesale_cost@13, ws_ext_list_price@14] │ [Stage 17] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 18] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 18] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] @@ -364,10 +364,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ws_bill_customer_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ws_bill_customer_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_bill_customer_sk, ws_ext_discount_amt, ws_ext_sales_price, ws_ext_wholesale_cost, ws_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -386,10 +386,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_customer_sk, cs_ext_discount_amt, cs_ext_sales_price, cs_ext_wholesale_cost, cs_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -408,10 +408,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_customer_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_customer_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk, ss_ext_discount_amt, ss_ext_sales_price, ss_ext_wholesale_cost, ss_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -430,10 +430,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_customer_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_customer_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk, ss_ext_discount_amt, ss_ext_sales_price, ss_ext_wholesale_cost, ss_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 13 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -452,10 +452,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 15 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 15 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_customer_sk, cs_ext_discount_amt, cs_ext_sales_price, cs_ext_wholesale_cost, cs_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 16 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -474,10 +474,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 18 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 18 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ws_bill_customer_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ws_bill_customer_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_bill_customer_sk, ws_ext_discount_amt, ws_ext_sales_price, ws_ext_wholesale_cost, ws_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── "); @@ -545,7 +545,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Left, on=[(wr_item_sk@1, ws_item_sk@0), (wr_order_number@2, ws_order_number@2)] │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, s_store_id@1 as s_store_id, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] @@ -618,10 +618,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_returns/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_returned_date_sk, wr_item_sk, wr_order_number, wr_return_amt, wr_net_loss], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ws_item_sk@0, ws_order_number@2], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ws_item_sk@0, ws_order_number@2], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_item_sk, ws_web_site_sk, ws_order_number], file_type=parquet └────────────────────────────────────────────────── "); @@ -739,8 +739,8 @@ mod tests { │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, ss_cdemo_sk@2)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_promo_sk@5, ss_quantity@6, ss_list_price@7, ss_sales_price@8, ss_coupon_amt@9] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── @@ -760,19 +760,19 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@1 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 3), input_partitions=3 + │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 3), input_partitions=2 │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, projection=[cd_demo_sk@0] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_gender, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, pruning_predicate=cd_gender_null_count@2 != row_count@3 AND cd_gender_min@0 <= M AND M <= cd_gender_max@1 AND cd_marital_status_null_count@6 != row_count@3 AND cd_marital_status_min@4 <= S AND S <= cd_marital_status_max@5 AND cd_education_status_null_count@9 != row_count@3 AND cd_education_status_min@7 <= College AND College <= cd_education_status_max@8, required_guarantees=[cd_education_status in (College), cd_gender in (M), cd_marital_status in (S)] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_cdemo_sk@2], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_cdemo_sk@2], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_cdemo_sk, ss_promo_sk, ss_quantity, ss_list_price, ss_sales_price, ss_coupon_amt], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── "); @@ -881,53 +881,53 @@ mod tests { │ ProjectionExec: expr=[count(Int64(1))@0 as count(*)] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] │ CoalescePartitionsExec - │ [Stage 3] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 3] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_net_paid)] │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 │ ProjectionExec: expr=[count(Int64(1))@0 as count(*)] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 5] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 5] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] │ CoalescePartitionsExec - │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_net_paid)] │ CoalescePartitionsExec - │ [Stage 7] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 7] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 │ ProjectionExec: expr=[count(Int64(1))@0 as count(*)] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 8] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 8] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] │ CoalescePartitionsExec - │ [Stage 9] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 9] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_net_paid)] │ CoalescePartitionsExec - │ [Stage 10] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 10] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 │ ProjectionExec: expr=[count(Int64(1))@0 as count(*)] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 11] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 11] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] │ CoalescePartitionsExec - │ [Stage 12] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 12] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_net_paid)] │ CoalescePartitionsExec - │ [Stage 13] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 13] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 │ ProjectionExec: expr=[count(Int64(1))@0 as count(*)] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 14] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 14] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] │ CoalescePartitionsExec - │ [Stage 15] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 15] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_net_paid)] │ CoalescePartitionsExec - │ [Stage 16] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 16] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] │ ProjectionExec: expr=[] @@ -937,114 +937,114 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/reason/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/reason/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/reason/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/reason/part-3.parquet]]}, projection=[r_reason_sk], file_type=parquet, predicate=r_reason_sk@0 = 1, pruning_predicate=r_reason_sk_null_count@2 != row_count@3 AND r_reason_sk_min@0 <= 1 AND 1 <= r_reason_sk_max@1, required_guarantees=[r_reason_sk in (1)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] │ ProjectionExec: expr=[] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ss_quantity@0 >= 1 AND ss_quantity@0 <= 20 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity], file_type=parquet, predicate=ss_quantity@0 >= 1 AND ss_quantity@0 <= 20, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 1 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 20, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 3 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ss_quantity@0 >= 1 AND ss_quantity@0 <= 20, projection=[ss_ext_discount_amt@1] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_ext_discount_amt], file_type=parquet, predicate=ss_quantity@0 >= 1 AND ss_quantity@0 <= 20, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 1 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 20, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_net_paid)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ss_quantity@0 >= 1 AND ss_quantity@0 <= 20, projection=[ss_net_paid@1] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_net_paid], file_type=parquet, predicate=ss_quantity@0 >= 1 AND ss_quantity@0 <= 20, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 1 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 20, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 5 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] │ ProjectionExec: expr=[] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ss_quantity@0 >= 21 AND ss_quantity@0 <= 40 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity], file_type=parquet, predicate=ss_quantity@0 >= 21 AND ss_quantity@0 <= 40, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 21 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 40, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 6 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ss_quantity@0 >= 21 AND ss_quantity@0 <= 40, projection=[ss_ext_discount_amt@1] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_ext_discount_amt], file_type=parquet, predicate=ss_quantity@0 >= 21 AND ss_quantity@0 <= 40, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 21 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 40, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 7 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_net_paid)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ss_quantity@0 >= 21 AND ss_quantity@0 <= 40, projection=[ss_net_paid@1] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_net_paid], file_type=parquet, predicate=ss_quantity@0 >= 21 AND ss_quantity@0 <= 40, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 21 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 40, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 8 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] │ ProjectionExec: expr=[] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ss_quantity@0 >= 41 AND ss_quantity@0 <= 60 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity], file_type=parquet, predicate=ss_quantity@0 >= 41 AND ss_quantity@0 <= 60, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 41 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 60, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 9 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ss_quantity@0 >= 41 AND ss_quantity@0 <= 60, projection=[ss_ext_discount_amt@1] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_ext_discount_amt], file_type=parquet, predicate=ss_quantity@0 >= 41 AND ss_quantity@0 <= 60, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 41 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 60, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 10 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_net_paid)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ss_quantity@0 >= 41 AND ss_quantity@0 <= 60, projection=[ss_net_paid@1] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_net_paid], file_type=parquet, predicate=ss_quantity@0 >= 41 AND ss_quantity@0 <= 60, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 41 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 60, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 11 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] │ ProjectionExec: expr=[] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ss_quantity@0 >= 61 AND ss_quantity@0 <= 80 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity], file_type=parquet, predicate=ss_quantity@0 >= 61 AND ss_quantity@0 <= 80, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 61 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 80, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 12 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ss_quantity@0 >= 61 AND ss_quantity@0 <= 80, projection=[ss_ext_discount_amt@1] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_ext_discount_amt], file_type=parquet, predicate=ss_quantity@0 >= 61 AND ss_quantity@0 <= 80, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 61 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 80, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 13 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_net_paid)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ss_quantity@0 >= 61 AND ss_quantity@0 <= 80, projection=[ss_net_paid@1] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_net_paid], file_type=parquet, predicate=ss_quantity@0 >= 61 AND ss_quantity@0 <= 80, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 61 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 80, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 14 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] │ ProjectionExec: expr=[] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ss_quantity@0 >= 81 AND ss_quantity@0 <= 100 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity], file_type=parquet, predicate=ss_quantity@0 >= 81 AND ss_quantity@0 <= 100, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 81 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 100, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 15 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 15 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ss_quantity@0 >= 81 AND ss_quantity@0 <= 100, projection=[ss_ext_discount_amt@1] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_ext_discount_amt], file_type=parquet, predicate=ss_quantity@0 >= 81 AND ss_quantity@0 <= 100, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 81 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 100, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 16 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_net_paid)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ss_quantity@0 >= 81 AND ss_quantity@0 <= 100, projection=[ss_net_paid@1] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_net_paid], file_type=parquet, predicate=ss_quantity@0 >= 81 AND ss_quantity@0 <= 100, pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 81 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 100, required_guarantees=[] └────────────────────────────────────────────────── "); @@ -1167,7 +1167,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, ws_bill_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, ws_sold_date_sk@9, ws_ext_discount_amt@11, ws_ext_list_price@12] │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)] │ CoalescePartitionsExec @@ -1187,7 +1187,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, ss_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, ss_sold_date_sk@9, ss_ext_discount_amt@11, ss_ext_list_price@12] │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[c_customer_id@0 as customer_id, c_first_name@1 as customer_first_name, c_last_name@2 as customer_last_name, c_preferred_cust_flag@3 as customer_preferred_cust_flag, sum(store_sales.ss_ext_list_price - store_sales.ss_ext_discount_amt)@8 as year_total] │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@7 as d_year], aggr=[sum(store_sales.ss_ext_list_price - store_sales.ss_ext_discount_amt)], ordering_mode=PartiallySorted([7]) │ CoalesceBatchesExec: target_batch_size=8192 @@ -1201,7 +1201,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, ss_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, ss_sold_date_sk@9, ss_ext_discount_amt@11, ss_ext_list_price@12] │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(web_sales.ws_ext_list_price - web_sales.ws_ext_discount_amt)@8 as year_total] │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@7 as d_year], aggr=[sum(web_sales.ws_ext_list_price - web_sales.ws_ext_discount_amt)], ordering_mode=PartiallySorted([7]) │ CoalesceBatchesExec: target_batch_size=8192 @@ -1215,7 +1215,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, ws_bill_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, ws_sold_date_sk@9, ws_ext_discount_amt@11, ws_ext_list_price@12] │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] @@ -1233,10 +1233,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ws_bill_customer_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ws_bill_customer_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_bill_customer_sk, ws_ext_discount_amt, ws_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -1255,10 +1255,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_customer_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_customer_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk, ss_ext_discount_amt, ss_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -1277,10 +1277,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_customer_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_customer_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk, ss_ext_discount_amt, ss_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -1299,10 +1299,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ws_bill_customer_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ws_bill_customer_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_bill_customer_sk, ws_ext_discount_amt, ws_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── "); @@ -1447,7 +1447,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, ss_sold_date_sk@4, ss_item_sk@5, ss_quantity@6, ss_list_price@7] │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[i_item_sk@0 as ss_item_sk] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(brand_id@0, i_brand_id@1), (class_id@1, i_class_id@2), (category_id@2, i_category_id@3)], projection=[i_item_sk@3] @@ -1463,7 +1463,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, ws_sold_date_sk@4] │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ AggregateExec: mode=SinglePartitioned, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(i_brand_id@0, brand_id@0), (i_class_id@1, class_id@1), (i_category_id@2, category_id@2)], NullsEqual: true @@ -1476,7 +1476,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, cs_sold_date_sk@4] │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 13] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 13] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ AggregateExec: mode=FinalPartitioned, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] │ CoalesceBatchesExec: target_batch_size=8192 │ RepartitionExec: partitioning=Hash([brand_id@0, class_id@1, category_id@2], 3), input_partitions=3 @@ -1490,7 +1490,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, ss_sold_date_sk@4] │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 16] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 16] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet, predicate=DynamicFilter [ empty ] │ ProjectionExec: expr=[catalog as channel, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, sum(catalog_sales.cs_quantity * catalog_sales.cs_list_price)@3 as sales, count(Int64(1))@4 as number_sales] @@ -1515,7 +1515,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, cs_sold_date_sk@4, cs_item_sk@5, cs_quantity@6, cs_list_price@7] │ [Stage 22] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 23] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 23] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[i_item_sk@0 as ss_item_sk] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(brand_id@0, i_brand_id@1), (class_id@1, i_class_id@2), (category_id@2, i_category_id@3)], projection=[i_item_sk@3] @@ -1531,7 +1531,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, ws_sold_date_sk@4] │ [Stage 25] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 26] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 26] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ AggregateExec: mode=SinglePartitioned, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(i_brand_id@0, brand_id@0), (i_class_id@1, class_id@1), (i_category_id@2, category_id@2)], NullsEqual: true @@ -1544,7 +1544,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, cs_sold_date_sk@4] │ [Stage 28] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 29] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 29] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ AggregateExec: mode=FinalPartitioned, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] │ CoalesceBatchesExec: target_batch_size=8192 │ RepartitionExec: partitioning=Hash([brand_id@0, class_id@1, category_id@2], 3), input_partitions=3 @@ -1558,7 +1558,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, ss_sold_date_sk@4] │ [Stage 31] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 32] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 32] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet, predicate=DynamicFilter [ empty ] │ ProjectionExec: expr=[web as channel, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, sum(web_sales.ws_quantity * web_sales.ws_list_price)@3 as sales, count(Int64(1))@4 as number_sales] @@ -1583,7 +1583,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, ws_sold_date_sk@4, ws_item_sk@5, ws_quantity@6, ws_list_price@7] │ [Stage 38] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 39] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 39] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[i_item_sk@0 as ss_item_sk] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(brand_id@0, i_brand_id@1), (class_id@1, i_class_id@2), (category_id@2, i_category_id@3)], projection=[i_item_sk@3] @@ -1599,7 +1599,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, ws_sold_date_sk@4] │ [Stage 41] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 42] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 42] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ AggregateExec: mode=SinglePartitioned, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(i_brand_id@0, brand_id@0), (i_class_id@1, class_id@1), (i_category_id@2, category_id@2)], NullsEqual: true @@ -1612,7 +1612,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, cs_sold_date_sk@4] │ [Stage 44] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 45] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 45] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ AggregateExec: mode=FinalPartitioned, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] │ CoalesceBatchesExec: target_batch_size=8192 │ RepartitionExec: partitioning=Hash([brand_id@0, class_id@1, category_id@2], 3), input_partitions=3 @@ -1626,7 +1626,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand_id@1, i_class_id@2, i_category_id@3, ss_sold_date_sk@4] │ [Stage 47] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 48] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 48] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── @@ -1691,10 +1691,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_item_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_item_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_quantity, ss_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -1712,10 +1712,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ws_item_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ws_item_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -1733,10 +1733,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 13 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_item_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([cs_item_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 14 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -1754,10 +1754,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 16 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_item_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_item_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 20 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] @@ -1821,10 +1821,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 23 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 23 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_item_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([cs_item_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_item_sk, cs_quantity, cs_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 24 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -1842,10 +1842,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 26 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 26 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ws_item_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ws_item_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 27 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -1863,10 +1863,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 29 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 29 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_item_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([cs_item_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 30 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -1884,10 +1884,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 32 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 32 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_item_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_item_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 36 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] @@ -1951,10 +1951,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 39 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 39 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ws_item_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ws_item_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_quantity, ws_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 40 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -1972,10 +1972,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 42 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 42 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ws_item_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ws_item_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 43 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -1993,10 +1993,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 45 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 45 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_item_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([cs_item_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 46 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -2014,10 +2014,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 48 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 48 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_item_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_item_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── "); @@ -2042,7 +2042,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@2)], filter=substr(ca_zip@2, 1, 5) IN ([85669, 86197, 88274, 83405, 86475, 85392, 85460, 80348, 81792]) OR ca_state@1 = CA OR ca_state@1 = WA OR ca_state@1 = GA OR cs_sales_price@0 > Some(50000),5,2, projection=[ca_zip@2, cs_sold_date_sk@3, cs_sales_price@4] │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] @@ -2059,27 +2059,27 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state, ca_zip], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 │ RepartitionExec: partitioning=Hash([c_current_addr_sk@2], 3), input_partitions=3 │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_sales_price@2 as cs_sales_price, c_current_addr_sk@0 as c_current_addr_sk] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@2, cs_bill_customer_sk@1)], projection=[c_current_addr_sk@1, cs_sold_date_sk@3, cs_sales_price@5] │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([CAST(customer.c_customer_sk AS Float64)@2], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([CAST(customer.c_customer_sk AS Float64)@2], 9), input_partitions=3 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@1], 6), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@1], 9), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_customer_sk, cs_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── "); @@ -2183,8 +2183,8 @@ mod tests { │ ProjectionExec: expr=[ss_sold_date_sk@2 as ss_sold_date_sk, ss_item_sk@3 as ss_item_sk, ss_store_sk@4 as ss_store_sk, ss_quantity@5 as ss_quantity, sr_returned_date_sk@6 as sr_returned_date_sk, sr_return_quantity@7 as sr_return_quantity, cs_sold_date_sk@0 as cs_sold_date_sk, cs_quantity@1 as cs_quantity] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(cs_bill_customer_sk@1, sr_customer_sk@6), (cs_item_sk@2, sr_item_sk@5)], projection=[cs_sold_date_sk@0, cs_quantity@3, ss_sold_date_sk@4, ss_item_sk@5, ss_store_sk@6, ss_quantity@7, sr_returned_date_sk@8, sr_return_quantity@11] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id, i_item_desc], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── @@ -2217,31 +2217,31 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_quarter_name], file_type=parquet, predicate=d_quarter_name@1 = 2001Q1, pruning_predicate=d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q1 AND 2001Q1 <= d_quarter_name_max@1, required_guarantees=[d_quarter_name in (2001Q1)] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@1, cs_item_sk@2], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@1, cs_item_sk@2], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_customer_sk, cs_item_sk, cs_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 │ RepartitionExec: partitioning=Hash([sr_customer_sk@6, sr_item_sk@5], 3), input_partitions=3 │ ProjectionExec: expr=[ss_sold_date_sk@4 as ss_sold_date_sk, ss_item_sk@5 as ss_item_sk, ss_store_sk@6 as ss_store_sk, ss_quantity@7 as ss_quantity, sr_returned_date_sk@0 as sr_returned_date_sk, sr_item_sk@1 as sr_item_sk, sr_customer_sk@2 as sr_customer_sk, sr_return_quantity@3 as sr_return_quantity] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_customer_sk@2, ss_customer_sk@2), (sr_item_sk@1, ss_item_sk@1), (sr_ticket_number@3, ss_ticket_number@4)], projection=[sr_returned_date_sk@0, sr_item_sk@1, sr_customer_sk@2, sr_return_quantity@4, ss_sold_date_sk@5, ss_item_sk@6, ss_store_sk@8, ss_quantity@10] - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([sr_customer_sk@2, sr_item_sk@1, sr_ticket_number@3], 6), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([sr_customer_sk@2, sr_item_sk@1, sr_ticket_number@3], 9), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_returned_date_sk, sr_item_sk, sr_customer_sk, sr_ticket_number, sr_return_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_customer_sk@2, ss_item_sk@1, ss_ticket_number@4], 6), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_customer_sk@2, ss_item_sk@1, ss_ticket_number@4], 9), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_store_sk, ss_ticket_number, ss_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── "); @@ -2283,9 +2283,9 @@ mod tests { │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_bill_customer_sk@2 as cs_bill_customer_sk, cs_item_sk@3 as cs_item_sk, cs_quantity@4 as cs_quantity, cs_list_price@5 as cs_list_price, cs_sales_price@6 as cs_sales_price, cs_coupon_amt@7 as cs_coupon_amt, cs_net_profit@8 as cs_net_profit, cd_dep_count@0 as cd_dep_count] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(cd1.cd_demo_sk AS Float64)@2, cs_bill_cdemo_sk@2)], projection=[cd_dep_count@1, cs_sold_date_sk@3, cs_bill_customer_sk@4, cs_item_sk@6, cs_quantity@7, cs_list_price@8, cs_sales_price@9, cs_coupon_amt@10, cs_net_profit@11] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── @@ -2312,26 +2312,26 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_addr_sk, c_birth_month, c_birth_year], file_type=parquet, predicate=c_birth_month@3 IN (SET) ([1, 6, 8, 9, 12, 2]) AND DynamicFilter [ empty ], pruning_predicate=c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 1 AND 1 <= c_birth_month_max@1 OR c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 6 AND 6 <= c_birth_month_max@1 OR c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 8 AND 8 <= c_birth_month_max@1 OR c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 9 AND 9 <= c_birth_month_max@1 OR c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 12 AND 12 <= c_birth_month_max@1 OR c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 2 AND 2 <= c_birth_month_max@1, required_guarantees=[c_birth_month in (1, 12, 2, 6, 8, 9)] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([CAST(cd1.cd_demo_sk AS Float64)@2], 3), input_partitions=3 + │ RepartitionExec: partitioning=Hash([CAST(cd1.cd_demo_sk AS Float64)@2], 3), input_partitions=2 │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, cd_dep_count@1 as cd_dep_count, CAST(cd_demo_sk@0 AS Float64) as CAST(cd1.cd_demo_sk AS Float64)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: cd_gender@1 = F AND cd_education_status@2 = Unknown, projection=[cd_demo_sk@0, cd_dep_count@3] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_gender, cd_education_status, cd_dep_count], file_type=parquet, predicate=cd_gender@1 = F AND cd_education_status@2 = Unknown, pruning_predicate=cd_gender_null_count@2 != row_count@3 AND cd_gender_min@0 <= F AND F <= cd_gender_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= Unknown AND Unknown <= cd_education_status_max@5, required_guarantees=[cd_education_status in (Unknown), cd_gender in (F)] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_bill_cdemo_sk@2], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([cs_bill_cdemo_sk@2], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_customer_sk, cs_bill_cdemo_sk, cs_item_sk, cs_quantity, cs_list_price, cs_sales_price, cs_coupon_amt, cs_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([CAST(cd2.cd_demo_sk AS Float64)@1], 3), input_partitions=3 + │ RepartitionExec: partitioning=Hash([CAST(cd2.cd_demo_sk AS Float64)@1], 3), input_partitions=2 │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, CAST(cd_demo_sk@0 AS Float64) as CAST(cd2.cd_demo_sk AS Float64)] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk], file_type=parquet └────────────────────────────────────────────────── "); @@ -2583,7 +2583,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@3, cs_bill_customer_sk@1)], projection=[c_first_name@1, c_last_name@2, cs_sold_date_sk@4, cs_bill_customer_sk@5, cs_item_sk@6, cs_quantity@7, cs_list_price@8] │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[i_item_sk@0 as item_sk] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: count(Int64(1))@1 > 4, projection=[i_item_sk@0] @@ -2632,12 +2632,12 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@1, ss_customer_sk@1)], projection=[c_customer_sk@0, ss_sold_date_sk@2, ss_quantity@4, ss_sales_price@5] │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[ss_quantity@1 as ss_quantity, ss_sales_price@2 as ss_sales_price, c_customer_sk@0 as c_customer_sk] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@1, ss_customer_sk@0)], projection=[c_customer_sk@0, ss_quantity@3, ss_sales_price@4] │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, sum(web_sales.ws_quantity * web_sales.ws_list_price)@2 as sales] │ AggregateExec: mode=FinalPartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name], aggr=[sum(web_sales.ws_quantity * web_sales.ws_list_price)] │ CoalesceBatchesExec: target_batch_size=8192 @@ -2658,7 +2658,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@3, ws_bill_customer_sk@2)], projection=[c_first_name@1, c_last_name@2, ws_sold_date_sk@4, ws_item_sk@5, ws_bill_customer_sk@6, ws_quantity@7, ws_list_price@8] │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[i_item_sk@0 as item_sk] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: count(Int64(1))@1 > 4, projection=[i_item_sk@0] @@ -2707,12 +2707,12 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@1, ss_customer_sk@1)], projection=[c_customer_sk@0, ss_sold_date_sk@2, ss_quantity@4, ss_sales_price@5] │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 16] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 16] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[ss_quantity@1 as ss_quantity, ss_sales_price@2 as ss_sales_price, c_customer_sk@0 as c_customer_sk] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@1, ss_customer_sk@0)], projection=[c_customer_sk@0, ss_quantity@3, ss_sales_price@4] │ [Stage 17] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 18] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 18] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] @@ -2730,10 +2730,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_first_name, c_last_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_customer_sk, cs_item_sk, cs_quantity, cs_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -2760,10 +2760,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_customer_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_customer_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk, ss_quantity, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] @@ -2774,10 +2774,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_customer_sk@0], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_customer_sk@0], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_customer_sk, ss_quantity, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -2796,10 +2796,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_first_name, c_last_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ws_bill_customer_sk@2], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ws_bill_customer_sk@2], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_bill_customer_sk, ws_quantity, ws_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 13 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -2826,10 +2826,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 16 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_customer_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_customer_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk, ss_quantity, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 17 ── Tasks: t0:[p0..p2] t1:[p0..p2] @@ -2840,10 +2840,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 18 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 18 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_customer_sk@0], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_customer_sk@0], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_customer_sk, ss_quantity, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── "); @@ -2884,8 +2884,8 @@ mod tests { │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_ticket_number@1, ss_ticket_number@3), (sr_item_sk@0, ss_item_sk@0)], projection=[ss_item_sk@2, ss_customer_sk@3, ss_store_sk@4, ss_net_paid@6] - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: i_color@3 = peach │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 @@ -2920,8 +2920,8 @@ mod tests { │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_ticket_number@1, ss_ticket_number@3), (sr_item_sk@0, ss_item_sk@0)], projection=[ss_item_sk@2, ss_customer_sk@3, ss_store_sk@4, ss_net_paid@6] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_current_price, i_size, i_color, i_units, i_manager_id], file_type=parquet, predicate=DynamicFilter [ empty ] │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_birth_country@4 as c_birth_country, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] @@ -2938,16 +2938,16 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_market_id, s_state, s_zip], file_type=parquet, predicate=s_market_id@2 = 8, pruning_predicate=s_market_id_null_count@2 != row_count@3 AND s_market_id_min@0 <= 8 AND 8 <= s_market_id_max@1, required_guarantees=[s_market_id in (8)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([sr_ticket_number@1, sr_item_sk@0], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([sr_ticket_number@1, sr_item_sk@0], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_item_sk, sr_ticket_number], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_ticket_number@3, ss_item_sk@0], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_ticket_number@3, ss_item_sk@0], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_item_sk, ss_customer_sk, ss_store_sk, ss_ticket_number, ss_net_paid], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -2958,16 +2958,16 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_market_id, s_state, s_zip], file_type=parquet, predicate=s_market_id@2 = 8, pruning_predicate=s_market_id_null_count@2 != row_count@3 AND s_market_id_min@0 <= 8 AND 8 <= s_market_id_max@1, required_guarantees=[s_market_id in (8)] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([sr_ticket_number@1, sr_item_sk@0], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([sr_ticket_number@1, sr_item_sk@0], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_item_sk, sr_ticket_number], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_ticket_number@3, ss_item_sk@0], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_ticket_number@3, ss_item_sk@0], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_item_sk, ss_customer_sk, ss_store_sk, ss_ticket_number, ss_net_paid], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── "); @@ -3004,8 +3004,8 @@ mod tests { │ ProjectionExec: expr=[ss_sold_date_sk@2 as ss_sold_date_sk, ss_item_sk@3 as ss_item_sk, ss_store_sk@4 as ss_store_sk, ss_net_profit@5 as ss_net_profit, sr_returned_date_sk@6 as sr_returned_date_sk, sr_net_loss@7 as sr_net_loss, cs_sold_date_sk@0 as cs_sold_date_sk, cs_net_profit@1 as cs_net_profit] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(cs_bill_customer_sk@1, sr_customer_sk@6), (cs_item_sk@2, sr_item_sk@5)], projection=[cs_sold_date_sk@0, cs_net_profit@3, ss_sold_date_sk@4, ss_item_sk@5, ss_store_sk@6, ss_net_profit@7, sr_returned_date_sk@8, sr_net_loss@11] - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: d_moy@2 >= 4 AND d_moy@2 <= 10 AND d_year@1 = 2001, projection=[d_date_sk@0] @@ -3029,31 +3029,31 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@2 = 4 AND d_year@1 = 2001, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 4 AND 4 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_moy in (4), d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@1, cs_item_sk@2], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@1, cs_item_sk@2], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_customer_sk, cs_item_sk, cs_net_profit], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 │ RepartitionExec: partitioning=Hash([sr_customer_sk@6, sr_item_sk@5], 3), input_partitions=3 │ ProjectionExec: expr=[ss_sold_date_sk@4 as ss_sold_date_sk, ss_item_sk@5 as ss_item_sk, ss_store_sk@6 as ss_store_sk, ss_net_profit@7 as ss_net_profit, sr_returned_date_sk@0 as sr_returned_date_sk, sr_item_sk@1 as sr_item_sk, sr_customer_sk@2 as sr_customer_sk, sr_net_loss@3 as sr_net_loss] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_customer_sk@2, ss_customer_sk@2), (sr_item_sk@1, ss_item_sk@1), (sr_ticket_number@3, ss_ticket_number@4)], projection=[sr_returned_date_sk@0, sr_item_sk@1, sr_customer_sk@2, sr_net_loss@4, ss_sold_date_sk@5, ss_item_sk@6, ss_store_sk@8, ss_net_profit@10] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([sr_customer_sk@2, sr_item_sk@1, sr_ticket_number@3], 6), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([sr_customer_sk@2, sr_item_sk@1, sr_ticket_number@3], 9), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_returned_date_sk, sr_item_sk, sr_customer_sk, sr_ticket_number, sr_net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_customer_sk@2, ss_item_sk@1, ss_ticket_number@4], 6), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_customer_sk@2, ss_item_sk@1, ss_ticket_number@4], 9), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_store_sk, ss_ticket_number, ss_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── "); @@ -3084,8 +3084,8 @@ mod tests { │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, cs_bill_cdemo_sk@1)], projection=[cs_sold_date_sk@2, cs_item_sk@4, cs_promo_sk@5, cs_quantity@6, cs_list_price@7, cs_sales_price@8, cs_coupon_amt@9] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── @@ -3105,19 +3105,19 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@1 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 3), input_partitions=3 + │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 3), input_partitions=2 │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, projection=[cd_demo_sk@0] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_gender, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, pruning_predicate=cd_gender_null_count@2 != row_count@3 AND cd_gender_min@0 <= M AND M <= cd_gender_max@1 AND cd_marital_status_null_count@6 != row_count@3 AND cd_marital_status_min@4 <= S AND S <= cd_marital_status_max@5 AND cd_education_status_null_count@9 != row_count@3 AND cd_education_status_min@7 <= College AND College <= cd_education_status_max@8, required_guarantees=[cd_education_status in (College), cd_gender in (M), cd_marital_status in (S)] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_bill_cdemo_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([cs_bill_cdemo_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_cdemo_sk, cs_item_sk, cs_promo_sk, cs_quantity, cs_list_price, cs_sales_price, cs_coupon_amt], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── "); @@ -3154,8 +3154,8 @@ mod tests { │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, ss_cdemo_sk@2)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_store_sk@5, ss_quantity@6, ss_list_price@7, ss_sales_price@8, ss_coupon_amt@9] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] │ ProjectionExec: expr=[i_item_id@0 as i_item_id, NULL as s_state, 1 as g_state, avg(results.agg1)@1 as agg1, avg(results.agg2)@2 as agg2, avg(results.agg3)@3 as agg3, avg(results.agg4)@4 as agg4] @@ -3177,8 +3177,8 @@ mod tests { │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, ss_cdemo_sk@2)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_store_sk@5, ss_quantity@6, ss_list_price@7, ss_sales_price@8, ss_coupon_amt@9] - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] │ ProjectionExec: expr=[NULL as i_item_id, NULL as s_state, 1 as g_state, avg(results.agg1)@0 as agg1, avg(results.agg2)@1 as agg2, avg(results.agg3)@2 as agg3, avg(results.agg4)@3 as agg4] @@ -3199,8 +3199,8 @@ mod tests { │ [Stage 10] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, ss_cdemo_sk@2)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_store_sk@5, ss_quantity@6, ss_list_price@7, ss_sales_price@8, ss_coupon_amt@9] - │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── @@ -3220,19 +3220,19 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@1 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 3), input_partitions=3 + │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 3), input_partitions=2 │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, projection=[cd_demo_sk@0] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_gender, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, pruning_predicate=cd_gender_null_count@2 != row_count@3 AND cd_gender_min@0 <= M AND M <= cd_gender_max@1 AND cd_marital_status_null_count@6 != row_count@3 AND cd_marital_status_min@4 <= S AND S <= cd_marital_status_max@5 AND cd_education_status_null_count@9 != row_count@3 AND cd_education_status_min@7 <= College AND College <= cd_education_status_max@8, required_guarantees=[cd_education_status in (College), cd_gender in (M), cd_marital_status in (S)] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_cdemo_sk@2], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_cdemo_sk@2], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_cdemo_sk, ss_store_sk, ss_quantity, ss_list_price, ss_sales_price, ss_coupon_amt], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -3251,19 +3251,19 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@1 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 3), input_partitions=3 + │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 3), input_partitions=2 │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, projection=[cd_demo_sk@0] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_gender, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, pruning_predicate=cd_gender_null_count@2 != row_count@3 AND cd_gender_min@0 <= M AND M <= cd_gender_max@1 AND cd_marital_status_null_count@6 != row_count@3 AND cd_marital_status_min@4 <= S AND S <= cd_marital_status_max@5 AND cd_education_status_null_count@9 != row_count@3 AND cd_education_status_min@7 <= College AND College <= cd_education_status_max@8, required_guarantees=[cd_education_status in (College), cd_gender in (M), cd_marital_status in (S)] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_cdemo_sk@2], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_cdemo_sk@2], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_cdemo_sk, ss_store_sk, ss_quantity, ss_list_price, ss_sales_price, ss_coupon_amt], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -3282,19 +3282,19 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@1 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 3), input_partitions=3 + │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 3), input_partitions=2 │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, projection=[cd_demo_sk@0] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_gender, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_gender@1 = M AND cd_marital_status@2 = S AND cd_education_status@3 = College, pruning_predicate=cd_gender_null_count@2 != row_count@3 AND cd_gender_min@0 <= M AND M <= cd_gender_max@1 AND cd_marital_status_null_count@6 != row_count@3 AND cd_marital_status_min@4 <= S AND S <= cd_marital_status_max@5 AND cd_education_status_null_count@9 != row_count@3 AND cd_education_status_min@7 <= College AND College <= cd_education_status_max@8, required_guarantees=[cd_education_status in (College), cd_gender in (M), cd_marital_status in (S)] └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_cdemo_sk@2], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_cdemo_sk@2], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_cdemo_sk, ss_store_sk, ss_quantity, ss_list_price, ss_sales_price, ss_coupon_amt], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── "); @@ -3311,75 +3311,75 @@ mod tests { │ ProjectionExec: expr=[avg(store_sales.ss_list_price)@0 as b6_lp, count(store_sales.ss_list_price)@1 as b6_cnt, count(DISTINCT store_sales.ss_list_price)@2 as b6_cntd] │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 │ ProjectionExec: expr=[b1_lp@3 as b1_lp, b1_cnt@4 as b1_cnt, b1_cntd@5 as b1_cntd, b2_lp@6 as b2_lp, b2_cnt@7 as b2_cnt, b2_cntd@8 as b2_cntd, b3_lp@9 as b3_lp, b3_cnt@10 as b3_cnt, b3_cntd@11 as b3_cntd, b4_lp@12 as b4_lp, b4_cnt@13 as b4_cnt, b4_cntd@14 as b4_cntd, b5_lp@0 as b5_lp, b5_cnt@1 as b5_cnt, b5_cntd@2 as b5_cntd] │ CrossJoinExec │ ProjectionExec: expr=[avg(store_sales.ss_list_price)@0 as b5_lp, count(store_sales.ss_list_price)@1 as b5_cnt, count(DISTINCT store_sales.ss_list_price)@2 as b5_cntd] │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 │ ProjectionExec: expr=[b1_lp@3 as b1_lp, b1_cnt@4 as b1_cnt, b1_cntd@5 as b1_cntd, b2_lp@6 as b2_lp, b2_cnt@7 as b2_cnt, b2_cntd@8 as b2_cntd, b3_lp@9 as b3_lp, b3_cnt@10 as b3_cnt, b3_cntd@11 as b3_cntd, b4_lp@0 as b4_lp, b4_cnt@1 as b4_cnt, b4_cntd@2 as b4_cntd] │ CrossJoinExec │ ProjectionExec: expr=[avg(store_sales.ss_list_price)@0 as b4_lp, count(store_sales.ss_list_price)@1 as b4_cnt, count(DISTINCT store_sales.ss_list_price)@2 as b4_cntd] │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] │ CoalescePartitionsExec - │ [Stage 3] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 3] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 │ ProjectionExec: expr=[b1_lp@3 as b1_lp, b1_cnt@4 as b1_cnt, b1_cntd@5 as b1_cntd, b2_lp@6 as b2_lp, b2_cnt@7 as b2_cnt, b2_cntd@8 as b2_cntd, b3_lp@0 as b3_lp, b3_cnt@1 as b3_cnt, b3_cntd@2 as b3_cntd] │ CrossJoinExec │ ProjectionExec: expr=[avg(store_sales.ss_list_price)@0 as b3_lp, count(store_sales.ss_list_price)@1 as b3_cnt, count(DISTINCT store_sales.ss_list_price)@2 as b3_cntd] │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 │ CrossJoinExec │ ProjectionExec: expr=[avg(store_sales.ss_list_price)@0 as b1_lp, count(store_sales.ss_list_price)@1 as b1_cnt, count(DISTINCT store_sales.ss_list_price)@2 as b1_cntd] │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] │ CoalescePartitionsExec - │ [Stage 5] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 5] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 │ ProjectionExec: expr=[avg(store_sales.ss_list_price)@0 as b2_lp, count(store_sales.ss_list_price)@1 as b2_cnt, count(DISTINCT store_sales.ss_list_price)@2 as b2_cntd] │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] │ CoalescePartitionsExec - │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ss_quantity@0 >= 26 AND ss_quantity@0 <= 30 AND (ss_list_price@2 >= Some(15400),5,2 AND ss_list_price@2 <= Some(16400),5,2 OR ss_coupon_amt@3 >= Some(732600),7,2 AND ss_coupon_amt@3 <= Some(832600),7,2 OR ss_wholesale_cost@1 >= Some(700),5,2 AND ss_wholesale_cost@1 <= Some(2700),5,2), projection=[ss_list_price@2] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_wholesale_cost, ss_list_price, ss_coupon_amt], file_type=parquet, predicate=ss_quantity@0 >= 26 AND ss_quantity@0 <= 30 AND (ss_list_price@2 >= Some(15400),5,2 AND ss_list_price@2 <= Some(16400),5,2 OR ss_coupon_amt@3 >= Some(732600),7,2 AND ss_coupon_amt@3 <= Some(832600),7,2 OR ss_wholesale_cost@1 >= Some(700),5,2 AND ss_wholesale_cost@1 <= Some(2700),5,2), pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 26 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 30 AND (ss_list_price_null_count@5 != row_count@2 AND ss_list_price_max@4 >= Some(15400),5,2 AND ss_list_price_null_count@5 != row_count@2 AND ss_list_price_min@6 <= Some(16400),5,2 OR ss_coupon_amt_null_count@8 != row_count@2 AND ss_coupon_amt_max@7 >= Some(732600),7,2 AND ss_coupon_amt_null_count@8 != row_count@2 AND ss_coupon_amt_min@9 <= Some(832600),7,2 OR ss_wholesale_cost_null_count@11 != row_count@2 AND ss_wholesale_cost_max@10 >= Some(700),5,2 AND ss_wholesale_cost_null_count@11 != row_count@2 AND ss_wholesale_cost_min@12 <= Some(2700),5,2), required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ss_quantity@0 >= 21 AND ss_quantity@0 <= 25 AND (ss_list_price@2 >= Some(12200),5,2 AND ss_list_price@2 <= Some(13200),5,2 OR ss_coupon_amt@3 >= Some(83600),7,2 AND ss_coupon_amt@3 <= Some(183600),7,2 OR ss_wholesale_cost@1 >= Some(1700),5,2 AND ss_wholesale_cost@1 <= Some(3700),5,2), projection=[ss_list_price@2] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_wholesale_cost, ss_list_price, ss_coupon_amt], file_type=parquet, predicate=ss_quantity@0 >= 21 AND ss_quantity@0 <= 25 AND (ss_list_price@2 >= Some(12200),5,2 AND ss_list_price@2 <= Some(13200),5,2 OR ss_coupon_amt@3 >= Some(83600),7,2 AND ss_coupon_amt@3 <= Some(183600),7,2 OR ss_wholesale_cost@1 >= Some(1700),5,2 AND ss_wholesale_cost@1 <= Some(3700),5,2), pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 21 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 25 AND (ss_list_price_null_count@5 != row_count@2 AND ss_list_price_max@4 >= Some(12200),5,2 AND ss_list_price_null_count@5 != row_count@2 AND ss_list_price_min@6 <= Some(13200),5,2 OR ss_coupon_amt_null_count@8 != row_count@2 AND ss_coupon_amt_max@7 >= Some(83600),7,2 AND ss_coupon_amt_null_count@8 != row_count@2 AND ss_coupon_amt_min@9 <= Some(183600),7,2 OR ss_wholesale_cost_null_count@11 != row_count@2 AND ss_wholesale_cost_max@10 >= Some(1700),5,2 AND ss_wholesale_cost_null_count@11 != row_count@2 AND ss_wholesale_cost_min@12 <= Some(3700),5,2), required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 3 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ss_quantity@0 >= 16 AND ss_quantity@0 <= 20 AND (ss_list_price@2 >= Some(13500),5,2 AND ss_list_price@2 <= Some(14500),5,2 OR ss_coupon_amt@3 >= Some(607100),7,2 AND ss_coupon_amt@3 <= Some(707100),7,2 OR ss_wholesale_cost@1 >= Some(3800),5,2 AND ss_wholesale_cost@1 <= Some(5800),5,2), projection=[ss_list_price@2] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_wholesale_cost, ss_list_price, ss_coupon_amt], file_type=parquet, predicate=ss_quantity@0 >= 16 AND ss_quantity@0 <= 20 AND (ss_list_price@2 >= Some(13500),5,2 AND ss_list_price@2 <= Some(14500),5,2 OR ss_coupon_amt@3 >= Some(607100),7,2 AND ss_coupon_amt@3 <= Some(707100),7,2 OR ss_wholesale_cost@1 >= Some(3800),5,2 AND ss_wholesale_cost@1 <= Some(5800),5,2), pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 16 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 20 AND (ss_list_price_null_count@5 != row_count@2 AND ss_list_price_max@4 >= Some(13500),5,2 AND ss_list_price_null_count@5 != row_count@2 AND ss_list_price_min@6 <= Some(14500),5,2 OR ss_coupon_amt_null_count@8 != row_count@2 AND ss_coupon_amt_max@7 >= Some(607100),7,2 AND ss_coupon_amt_null_count@8 != row_count@2 AND ss_coupon_amt_min@9 <= Some(707100),7,2 OR ss_wholesale_cost_null_count@11 != row_count@2 AND ss_wholesale_cost_max@10 >= Some(3800),5,2 AND ss_wholesale_cost_null_count@11 != row_count@2 AND ss_wholesale_cost_min@12 <= Some(5800),5,2), required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ss_quantity@0 >= 11 AND ss_quantity@0 <= 15 AND (ss_list_price@2 >= Some(14200),5,2 AND ss_list_price@2 <= Some(15200),5,2 OR ss_coupon_amt@3 >= Some(1221400),7,2 AND ss_coupon_amt@3 <= Some(1321400),7,2 OR ss_wholesale_cost@1 >= Some(7900),5,2 AND ss_wholesale_cost@1 <= Some(9900),5,2), projection=[ss_list_price@2] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_wholesale_cost, ss_list_price, ss_coupon_amt], file_type=parquet, predicate=ss_quantity@0 >= 11 AND ss_quantity@0 <= 15 AND (ss_list_price@2 >= Some(14200),5,2 AND ss_list_price@2 <= Some(15200),5,2 OR ss_coupon_amt@3 >= Some(1221400),7,2 AND ss_coupon_amt@3 <= Some(1321400),7,2 OR ss_wholesale_cost@1 >= Some(7900),5,2 AND ss_wholesale_cost@1 <= Some(9900),5,2), pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 11 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 15 AND (ss_list_price_null_count@5 != row_count@2 AND ss_list_price_max@4 >= Some(14200),5,2 AND ss_list_price_null_count@5 != row_count@2 AND ss_list_price_min@6 <= Some(15200),5,2 OR ss_coupon_amt_null_count@8 != row_count@2 AND ss_coupon_amt_max@7 >= Some(1221400),7,2 AND ss_coupon_amt_null_count@8 != row_count@2 AND ss_coupon_amt_min@9 <= Some(1321400),7,2 OR ss_wholesale_cost_null_count@11 != row_count@2 AND ss_wholesale_cost_max@10 >= Some(7900),5,2 AND ss_wholesale_cost_null_count@11 != row_count@2 AND ss_wholesale_cost_min@12 <= Some(9900),5,2), required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 5 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ss_quantity@0 >= 0 AND ss_quantity@0 <= 5 AND (ss_list_price@2 >= Some(800),5,2 AND ss_list_price@2 <= Some(1800),5,2 OR ss_coupon_amt@3 >= Some(45900),7,2 AND ss_coupon_amt@3 <= Some(145900),7,2 OR ss_wholesale_cost@1 >= Some(5700),5,2 AND ss_wholesale_cost@1 <= Some(7700),5,2), projection=[ss_list_price@2] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_wholesale_cost, ss_list_price, ss_coupon_amt], file_type=parquet, predicate=ss_quantity@0 >= 0 AND ss_quantity@0 <= 5 AND (ss_list_price@2 >= Some(800),5,2 AND ss_list_price@2 <= Some(1800),5,2 OR ss_coupon_amt@3 >= Some(45900),7,2 AND ss_coupon_amt@3 <= Some(145900),7,2 OR ss_wholesale_cost@1 >= Some(5700),5,2 AND ss_wholesale_cost@1 <= Some(7700),5,2), pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 0 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 5 AND (ss_list_price_null_count@5 != row_count@2 AND ss_list_price_max@4 >= Some(800),5,2 AND ss_list_price_null_count@5 != row_count@2 AND ss_list_price_min@6 <= Some(1800),5,2 OR ss_coupon_amt_null_count@8 != row_count@2 AND ss_coupon_amt_max@7 >= Some(45900),7,2 AND ss_coupon_amt_null_count@8 != row_count@2 AND ss_coupon_amt_min@9 <= Some(145900),7,2 OR ss_wholesale_cost_null_count@11 != row_count@2 AND ss_wholesale_cost_max@10 >= Some(5700),5,2 AND ss_wholesale_cost_null_count@11 != row_count@2 AND ss_wholesale_cost_min@12 <= Some(7700),5,2), required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 6 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ss_quantity@0 >= 6 AND ss_quantity@0 <= 10 AND (ss_list_price@2 >= Some(9000),5,2 AND ss_list_price@2 <= Some(10000),5,2 OR ss_coupon_amt@3 >= Some(232300),7,2 AND ss_coupon_amt@3 <= Some(332300),7,2 OR ss_wholesale_cost@1 >= Some(3100),5,2 AND ss_wholesale_cost@1 <= Some(5100),5,2), projection=[ss_list_price@2] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_quantity, ss_wholesale_cost, ss_list_price, ss_coupon_amt], file_type=parquet, predicate=ss_quantity@0 >= 6 AND ss_quantity@0 <= 10 AND (ss_list_price@2 >= Some(9000),5,2 AND ss_list_price@2 <= Some(10000),5,2 OR ss_coupon_amt@3 >= Some(232300),7,2 AND ss_coupon_amt@3 <= Some(332300),7,2 OR ss_wholesale_cost@1 >= Some(3100),5,2 AND ss_wholesale_cost@1 <= Some(5100),5,2), pruning_predicate=ss_quantity_null_count@1 != row_count@2 AND ss_quantity_max@0 >= 6 AND ss_quantity_null_count@1 != row_count@2 AND ss_quantity_min@3 <= 10 AND (ss_list_price_null_count@5 != row_count@2 AND ss_list_price_max@4 >= Some(9000),5,2 AND ss_list_price_null_count@5 != row_count@2 AND ss_list_price_min@6 <= Some(10000),5,2 OR ss_coupon_amt_null_count@8 != row_count@2 AND ss_coupon_amt_max@7 >= Some(232300),7,2 AND ss_coupon_amt_null_count@8 != row_count@2 AND ss_coupon_amt_min@9 <= Some(332300),7,2 OR ss_wholesale_cost_null_count@11 != row_count@2 AND ss_wholesale_cost_max@10 >= Some(3100),5,2 AND ss_wholesale_cost_null_count@11 != row_count@2 AND ss_wholesale_cost_min@12 <= Some(5100),5,2), required_guarantees=[] └────────────────────────────────────────────────── "); @@ -3417,8 +3417,8 @@ mod tests { │ ProjectionExec: expr=[ss_sold_date_sk@2 as ss_sold_date_sk, ss_item_sk@3 as ss_item_sk, ss_store_sk@4 as ss_store_sk, ss_quantity@5 as ss_quantity, sr_returned_date_sk@6 as sr_returned_date_sk, sr_return_quantity@7 as sr_return_quantity, cs_sold_date_sk@0 as cs_sold_date_sk, cs_quantity@1 as cs_quantity] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(cs_bill_customer_sk@1, sr_customer_sk@6), (cs_item_sk@2, sr_item_sk@5)], projection=[cs_sold_date_sk@0, cs_quantity@3, ss_sold_date_sk@4, ss_item_sk@5, ss_store_sk@6, ss_quantity@7, sr_returned_date_sk@8, sr_return_quantity@11] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: d_year@1 = 1999 OR d_year@1 = 2000 OR d_year@1 = 2001, projection=[d_date_sk@0] @@ -3445,31 +3445,31 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@2 = 9 AND d_year@1 = 1999, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 9 AND 9 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5, required_guarantees=[d_moy in (9), d_year in (1999)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@1, cs_item_sk@2], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@1, cs_item_sk@2], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_customer_sk, cs_item_sk, cs_quantity], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 │ RepartitionExec: partitioning=Hash([sr_customer_sk@6, sr_item_sk@5], 3), input_partitions=3 │ ProjectionExec: expr=[ss_sold_date_sk@4 as ss_sold_date_sk, ss_item_sk@5 as ss_item_sk, ss_store_sk@6 as ss_store_sk, ss_quantity@7 as ss_quantity, sr_returned_date_sk@0 as sr_returned_date_sk, sr_item_sk@1 as sr_item_sk, sr_customer_sk@2 as sr_customer_sk, sr_return_quantity@3 as sr_return_quantity] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_customer_sk@2, ss_customer_sk@2), (sr_item_sk@1, ss_item_sk@1), (sr_ticket_number@3, ss_ticket_number@4)], projection=[sr_returned_date_sk@0, sr_item_sk@1, sr_customer_sk@2, sr_return_quantity@4, ss_sold_date_sk@5, ss_item_sk@6, ss_store_sk@8, ss_quantity@10] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([sr_customer_sk@2, sr_item_sk@1, sr_ticket_number@3], 6), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([sr_customer_sk@2, sr_item_sk@1, sr_ticket_number@3], 9), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_returned_date_sk, sr_item_sk, sr_customer_sk, sr_ticket_number, sr_return_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_customer_sk@2, ss_item_sk@1, ss_ticket_number@4], 6), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_customer_sk@2, ss_item_sk@1, ss_ticket_number@4], 9), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_store_sk, ss_ticket_number, ss_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── "); @@ -4177,7 +4177,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(inv_item_sk@1, i_item_sk@0)], projection=[inv_date_sk@0, i_item_sk@2, i_item_id@3, i_item_desc@4, i_current_price@5] │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: i_current_price@3 >= Some(6800),4,2 AND i_current_price@3 <= Some(9800),4,2 AND i_manufact_id@4 IN (SET) ([677, 940, 694, 808]), projection=[i_item_sk@0, i_item_id@1, i_item_desc@2, i_current_price@3] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 @@ -4188,10 +4188,10 @@ mod tests { │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@1 >= 2000-02-01 AND d_date@1 <= 2000-04-01 AND DynamicFilter [ empty ], pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-02-01 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-01, required_guarantees=[] │ DataSourceExec: file_groups={3 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:.., /testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-3.parquet:..]]}, projection=[cs_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: inv_quantity_on_hand@2 >= 100 AND inv_quantity_on_hand@2 <= 500, projection=[inv_date_sk@0, inv_item_sk@1] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/inventory/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], ...]}, projection=[inv_date_sk, inv_item_sk, inv_quantity_on_hand], file_type=parquet, predicate=inv_quantity_on_hand@2 >= 100 AND inv_quantity_on_hand@2 <= 500, pruning_predicate=inv_quantity_on_hand_null_count@1 != row_count@2 AND inv_quantity_on_hand_max@0 >= 100 AND inv_quantity_on_hand_null_count@1 != row_count@2 AND inv_quantity_on_hand_min@3 <= 500, required_guarantees=[] └────────────────────────────────────────────────── "); @@ -4329,7 +4329,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, inv_item_sk@1)], projection=[i_item_sk@0, inv_date_sk@1, inv_warehouse_sk@3, inv_quantity_on_hand@4] │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ CoalesceBatchesExec: target_batch_size=8192 │ RepartitionExec: partitioning=Hash([i_item_sk@1, w_warehouse_sk@0], 3), input_partitions=3 │ ProjectionExec: expr=[w_warehouse_sk@0 as w_warehouse_sk, i_item_sk@1 as i_item_sk, d_moy@2 as d_moy, avg(inventory.inv_quantity_on_hand)@4 as mean, CASE avg(inventory.inv_quantity_on_hand)@4 WHEN 0 THEN NULL ELSE stddev(inventory.inv_quantity_on_hand)@3 / avg(inventory.inv_quantity_on_hand)@4 END as cov] @@ -4354,7 +4354,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, inv_item_sk@1)], projection=[i_item_sk@0, inv_date_sk@1, inv_warehouse_sk@3, inv_quantity_on_hand@4] │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] │ CoalesceBatchesExec: target_batch_size=8192 @@ -4370,10 +4370,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([inv_item_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([inv_item_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/inventory/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], ...]}, projection=[inv_date_sk, inv_item_sk, inv_warehouse_sk, inv_quantity_on_hand], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -4390,10 +4390,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([inv_item_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([inv_item_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/inventory/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], ...]}, projection=[inv_date_sk, inv_item_sk, inv_warehouse_sk, inv_quantity_on_hand], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── "); @@ -4426,8 +4426,8 @@ mod tests { │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_warehouse_sk@2 as cs_warehouse_sk, cs_item_sk@3 as cs_item_sk, cs_sales_price@4 as cs_sales_price, cr_refunded_cash@0 as cr_refunded_cash] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(cr_order_number@1, cs_order_number@3), (cr_item_sk@0, cs_item_sk@2)], projection=[cr_refunded_cash@2, cs_sold_date_sk@3, cs_warehouse_sk@4, cs_item_sk@5, cs_sales_price@7] - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: i_current_price@2 >= Some(99),4,2 AND i_current_price@2 <= Some(149),4,2, projection=[i_item_sk@0, i_item_id@1] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 @@ -4443,16 +4443,16 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_state], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cr_order_number@1, cr_item_sk@0], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([cr_order_number@1, cr_item_sk@0], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_item_sk, cr_order_number, cr_refunded_cash], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_order_number@3, cs_item_sk@2], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([cs_order_number@3, cs_item_sk@2], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_warehouse_sk, cs_item_sk, cs_order_number, cs_sales_price], file_type=parquet └────────────────────────────────────────────────── "#); @@ -4619,10 +4619,10 @@ mod tests { │ NestedLoopJoinExec: join_type=Left │ CoalescePartitionsExec │ AggregateExec: mode=FinalPartitioned, gby=[ss_item_sk@0 as ss_item_sk], aggr=[avg(ss1.ss_net_profit)] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[CAST(avg(store_sales.ss_net_profit)@1 AS Float64) as rank_col] │ AggregateExec: mode=FinalPartitioned, gby=[ss_store_sk@0 as ss_store_sk], aggr=[avg(store_sales.ss_net_profit)], ordering_mode=Sorted - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ CoalesceBatchesExec: target_batch_size=8192 │ RepartitionExec: partitioning=Hash([rnk@1], 3), input_partitions=3 │ ProjectionExec: expr=[item_sk@0 as item_sk, rank() ORDER BY [v2.rank_col DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rnk] @@ -4639,45 +4639,45 @@ mod tests { │ NestedLoopJoinExec: join_type=Left │ CoalescePartitionsExec │ AggregateExec: mode=FinalPartitioned, gby=[ss_item_sk@0 as ss_item_sk], aggr=[avg(ss1.ss_net_profit)] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[CAST(avg(store_sales.ss_net_profit)@1 AS Float64) as rank_col] │ AggregateExec: mode=FinalPartitioned, gby=[ss_store_sk@0 as ss_store_sk], aggr=[avg(store_sales.ss_net_profit)], ordering_mode=Sorted - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_item_sk@0], 3), input_partitions=3 + │ RepartitionExec: partitioning=Hash([ss_item_sk@0], 3), input_partitions=2 │ AggregateExec: mode=Partial, gby=[ss_item_sk@0 as ss_item_sk], aggr=[avg(ss1.ss_net_profit)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ss_store_sk@1 = 4, projection=[ss_item_sk@0, ss_net_profit@2] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_item_sk, ss_store_sk, ss_net_profit], file_type=parquet, predicate=ss_store_sk@1 = 4, pruning_predicate=ss_store_sk_null_count@2 != row_count@3 AND ss_store_sk_min@0 <= 4 AND 4 <= ss_store_sk_max@1, required_guarantees=[ss_store_sk in (4)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_store_sk@0], 3), input_partitions=3 + │ RepartitionExec: partitioning=Hash([ss_store_sk@0], 3), input_partitions=2 │ AggregateExec: mode=Partial, gby=[ss_store_sk@0 as ss_store_sk], aggr=[avg(store_sales.ss_net_profit)], ordering_mode=Sorted │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ss_store_sk@1 = 4 AND ss_addr_sk@0 IS NULL, projection=[ss_store_sk@1, ss_net_profit@2] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_addr_sk, ss_store_sk, ss_net_profit], file_type=parquet, predicate=ss_store_sk@1 = 4 AND ss_addr_sk@0 IS NULL, pruning_predicate=ss_store_sk_null_count@2 != row_count@3 AND ss_store_sk_min@0 <= 4 AND 4 <= ss_store_sk_max@1 AND ss_addr_sk_null_count@4 > 0, required_guarantees=[ss_store_sk in (4)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_item_sk@0], 3), input_partitions=3 + │ RepartitionExec: partitioning=Hash([ss_item_sk@0], 3), input_partitions=2 │ AggregateExec: mode=Partial, gby=[ss_item_sk@0 as ss_item_sk], aggr=[avg(ss1.ss_net_profit)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ss_store_sk@1 = 4, projection=[ss_item_sk@0, ss_net_profit@2] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_item_sk, ss_store_sk, ss_net_profit], file_type=parquet, predicate=ss_store_sk@1 = 4, pruning_predicate=ss_store_sk_null_count@2 != row_count@3 AND ss_store_sk_min@0 <= 4 AND 4 <= ss_store_sk_max@1, required_guarantees=[ss_store_sk in (4)] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_store_sk@0], 3), input_partitions=3 + │ RepartitionExec: partitioning=Hash([ss_store_sk@0], 3), input_partitions=2 │ AggregateExec: mode=Partial, gby=[ss_store_sk@0 as ss_store_sk], aggr=[avg(store_sales.ss_net_profit)], ordering_mode=Sorted │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ss_store_sk@1 = 4 AND ss_addr_sk@0 IS NULL, projection=[ss_store_sk@1, ss_net_profit@2] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_addr_sk, ss_store_sk, ss_net_profit], file_type=parquet, predicate=ss_store_sk@1 = 4 AND ss_addr_sk@0 IS NULL, pruning_predicate=ss_store_sk_null_count@2 != row_count@3 AND ss_store_sk_min@0 <= 4 AND 4 <= ss_store_sk_max@1 AND ss_addr_sk_null_count@4 > 0, required_guarantees=[ss_store_sk in (4)] └────────────────────────────────────────────────── ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] @@ -4724,7 +4724,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@3)], projection=[ca_city@1, ca_zip@2, ws_sold_date_sk@3, ws_item_sk@4, ws_sales_price@5] │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet, predicate=DynamicFilter [ empty ] │ CoalesceBatchesExec: target_batch_size=8192 @@ -4747,27 +4747,27 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_city, ca_zip], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 │ RepartitionExec: partitioning=Hash([c_current_addr_sk@3], 3), input_partitions=3 │ ProjectionExec: expr=[ws_sold_date_sk@1 as ws_sold_date_sk, ws_item_sk@2 as ws_item_sk, ws_sales_price@3 as ws_sales_price, c_current_addr_sk@0 as c_current_addr_sk] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@2, ws_bill_customer_sk@2)], projection=[c_current_addr_sk@1, ws_sold_date_sk@3, ws_item_sk@4, ws_sales_price@6] │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([CAST(customer.c_customer_sk AS Float64)@2], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([CAST(customer.c_customer_sk AS Float64)@2], 9), input_partitions=3 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ws_bill_customer_sk@2], 6), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ws_bill_customer_sk@2], 9), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_bill_customer_sk, ws_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── "); @@ -4887,7 +4887,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand@1, i_category@2, ss_sold_date_sk@3, ss_store_sk@5, ss_sales_price@6] │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[i_category@0 as i_category, i_brand@1 as i_brand, s_store_name@2 as s_store_name, s_company_name@3 as s_company_name, sum(store_sales.ss_sales_price)@6 as sum_sales, rank() PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 as rn, CAST(rank() PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 AS Decimal128(20, 0)) + Some(1),20,0 as v1_lag.rn + Decimal128(Some(1),20,0)] │ BoundedWindowAggExec: wdw=[rank() PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] │ SortExec: expr=[i_category@0 ASC NULLS LAST, i_brand@1 ASC NULLS LAST, s_store_name@2 ASC NULLS LAST, s_company_name@3 ASC NULLS LAST, d_year@4 ASC NULLS LAST, d_moy@5 ASC NULLS LAST], preserve_partitioning=[true] @@ -4910,7 +4910,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand@1, i_category@2, ss_sold_date_sk@3, ss_store_sk@5, ss_sales_price@6] │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[i_category@0 as i_category, i_brand@1 as i_brand, s_store_name@2 as s_store_name, s_company_name@3 as s_company_name, sum(store_sales.ss_sales_price)@6 as sum_sales, rank() PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 as rn, CAST(rank() PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 AS Decimal128(20, 0)) - Some(1),20,0 as v1_lead.rn - Decimal128(Some(1),20,0)] │ BoundedWindowAggExec: wdw=[rank() PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] │ SortExec: expr=[i_category@0 ASC NULLS LAST, i_brand@1 ASC NULLS LAST, s_store_name@2 ASC NULLS LAST, s_company_name@3 ASC NULLS LAST, d_year@4 ASC NULLS LAST, d_moy@5 ASC NULLS LAST], preserve_partitioning=[true] @@ -4933,7 +4933,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand@1, i_category@2, ss_sold_date_sk@3, ss_store_sk@5, ss_sales_price@6] │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, s_store_name@1 as s_store_name, s_company_name@2 as s_company_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] @@ -4955,10 +4955,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_item_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_item_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p1] t1:[p2..p3] @@ -4981,10 +4981,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_item_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_item_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 9 ── Tasks: t0:[p0..p1] t1:[p2..p3] @@ -5007,10 +5007,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_item_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_item_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── "#); @@ -5102,7 +5102,7 @@ mod tests { │ FilterExec: wr_return_amt@5 > Some(1000000),7,2 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Left, on=[(ws_order_number@2, wr_order_number@1), (ws_item_sk@1, wr_item_sk@0)], projection=[ws_sold_date_sk@0, ws_item_sk@1, ws_quantity@3, ws_net_paid@4, wr_return_quantity@7, wr_return_amt@8] - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 │ ProjectionExec: expr=[catalog as channel, item@0 as item, return_ratio@1 as return_ratio, return_rank@2 as return_rank, currency_rank@3 as currency_rank] │ ProjectionExec: expr=[item@0 as item, return_ratio@1 as return_ratio, rank() ORDER BY [in_cat.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as return_rank, rank() ORDER BY [in_cat.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as currency_rank] @@ -5128,8 +5128,8 @@ mod tests { │ FilterExec: cr_return_amount@5 > Some(1000000),7,2 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Left, on=[(cs_order_number@2, cr_order_number@1), (cs_item_sk@1, cr_item_sk@0)], projection=[cs_sold_date_sk@0, cs_item_sk@1, cs_quantity@3, cs_net_paid@4, cr_return_quantity@7, cr_return_amount@8] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[store as channel, item@0 as item, return_ratio@1 as return_ratio, return_rank@2 as return_rank, currency_rank@3 as currency_rank] │ ProjectionExec: expr=[item@0 as item, return_ratio@1 as return_ratio, rank() ORDER BY [in_store.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as return_rank, rank() ORDER BY [in_store.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as currency_rank] │ CoalesceBatchesExec: target_batch_size=8192 @@ -5155,8 +5155,8 @@ mod tests { │ ProjectionExec: expr=[ss_sold_date_sk@2 as ss_sold_date_sk, ss_item_sk@3 as ss_item_sk, ss_quantity@4 as ss_quantity, ss_net_paid@5 as ss_net_paid, sr_return_quantity@0 as sr_return_quantity, sr_return_amt@1 as sr_return_amt] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(sr_ticket_number@1, ss_ticket_number@2), (sr_item_sk@0, ss_item_sk@1)], projection=[sr_return_quantity@2, sr_return_amt@3, ss_sold_date_sk@4, ss_item_sk@5, ss_quantity@7, ss_net_paid@8] - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] @@ -5166,12 +5166,12 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@1 = 2001 AND d_moy@2 = 12, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5, required_guarantees=[d_moy in (12), d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ws_order_number@2, ws_item_sk@1], 3), input_partitions=3 + │ RepartitionExec: partitioning=Hash([ws_order_number@2, ws_item_sk@1], 3), input_partitions=2 │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ws_net_profit@5 > Some(100),7,2 AND ws_net_paid@4 > Some(0),7,2 AND ws_quantity@3 > 0, projection=[ws_sold_date_sk@0, ws_item_sk@1, ws_order_number@2, ws_quantity@3, ws_net_paid@4] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_order_number, ws_quantity, ws_net_paid, ws_net_profit], file_type=parquet, predicate=ws_net_profit@5 > Some(100),7,2 AND ws_net_paid@4 > Some(0),7,2 AND ws_quantity@3 > 0, pruning_predicate=ws_net_profit_null_count@1 != row_count@2 AND ws_net_profit_max@0 > Some(100),7,2 AND ws_net_paid_null_count@4 != row_count@2 AND ws_net_paid_max@3 > Some(0),7,2 AND ws_quantity_null_count@6 != row_count@2 AND ws_quantity_max@5 > 0, required_guarantees=[] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] @@ -5189,18 +5189,18 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@1 = 2001 AND d_moy@2 = 12, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5, required_guarantees=[d_moy in (12), d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_order_number@2, cs_item_sk@1], 3), input_partitions=3 + │ RepartitionExec: partitioning=Hash([cs_order_number@2, cs_item_sk@1], 3), input_partitions=2 │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: cs_net_profit@5 > Some(100),7,2 AND cs_net_paid@4 > Some(0),7,2 AND cs_quantity@3 > 0, projection=[cs_sold_date_sk@0, cs_item_sk@1, cs_order_number@2, cs_quantity@3, cs_net_paid@4] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_item_sk, cs_order_number, cs_quantity, cs_net_paid, cs_net_profit], file_type=parquet, predicate=cs_net_profit@5 > Some(100),7,2 AND cs_net_paid@4 > Some(0),7,2 AND cs_quantity@3 > 0, pruning_predicate=cs_net_profit_null_count@1 != row_count@2 AND cs_net_profit_max@0 > Some(100),7,2 AND cs_net_paid_null_count@4 != row_count@2 AND cs_net_paid_max@3 > Some(0),7,2 AND cs_quantity_null_count@6 != row_count@2 AND cs_quantity_max@5 > 0, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cr_order_number@1, cr_item_sk@0], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([cr_order_number@1, cr_item_sk@0], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_item_sk, cr_order_number, cr_return_quantity, cr_return_amount], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -5211,18 +5211,18 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@1 = 2001 AND d_moy@2 = 12, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5, required_guarantees=[d_moy in (12), d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([sr_ticket_number@1, sr_item_sk@0], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([sr_ticket_number@1, sr_item_sk@0], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_item_sk, sr_ticket_number, sr_return_quantity, sr_return_amt], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_ticket_number@2, ss_item_sk@1], 3), input_partitions=3 + │ RepartitionExec: partitioning=Hash([ss_ticket_number@2, ss_item_sk@1], 3), input_partitions=2 │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ss_net_profit@5 > Some(100),6,2 AND ss_net_paid@4 > Some(0),7,2 AND ss_quantity@3 > 0, projection=[ss_sold_date_sk@0, ss_item_sk@1, ss_ticket_number@2, ss_quantity@3, ss_net_paid@4] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_ticket_number, ss_quantity, ss_net_paid, ss_net_profit], file_type=parquet, predicate=ss_net_profit@5 > Some(100),6,2 AND ss_net_paid@4 > Some(0),7,2 AND ss_quantity@3 > 0, pruning_predicate=ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_max@0 > Some(100),6,2 AND ss_net_paid_null_count@4 != row_count@2 AND ss_net_paid_max@3 > Some(0),7,2 AND ss_quantity_null_count@6 != row_count@2 AND ss_quantity_max@5 > 0, required_guarantees=[] └────────────────────────────────────────────────── "#); @@ -5255,8 +5255,8 @@ mod tests { │ ProjectionExec: expr=[ss_sold_date_sk@1 as ss_sold_date_sk, ss_store_sk@2 as ss_store_sk, sr_returned_date_sk@0 as sr_returned_date_sk] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_ticket_number@3, ss_ticket_number@4), (sr_item_sk@1, ss_item_sk@1), (sr_customer_sk@2, ss_customer_sk@2)], projection=[sr_returned_date_sk@0, ss_sold_date_sk@4, ss_store_sk@7] - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk], file_type=parquet @@ -5271,16 +5271,16 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_company_id, s_street_number, s_street_name, s_street_type, s_suite_number, s_city, s_county, s_state, s_zip], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([sr_ticket_number@3, sr_item_sk@1, sr_customer_sk@2], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([sr_ticket_number@3, sr_item_sk@1, sr_customer_sk@2], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_returned_date_sk, sr_item_sk, sr_customer_sk, sr_ticket_number], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_ticket_number@4, ss_item_sk@1, ss_customer_sk@2], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_ticket_number@4, ss_item_sk@1, ss_customer_sk@2], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_store_sk, ss_ticket_number], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── "); @@ -5777,7 +5777,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@2)], projection=[i_brand@1, i_category@2, cs_sold_date_sk@3, cs_call_center_sk@4, cs_sales_price@6] │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[i_category@0 as i_category, i_brand@1 as i_brand, cc_name@2 as cc_name, sum(catalog_sales.cs_sales_price)@5 as sum_sales, rank() PARTITION BY [item.i_category, item.i_brand, call_center.cc_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as rn, CAST(rank() PARTITION BY [item.i_category, item.i_brand, call_center.cc_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 AS Decimal128(20, 0)) + Some(1),20,0 as v1_lag.rn + Decimal128(Some(1),20,0)] │ BoundedWindowAggExec: wdw=[rank() PARTITION BY [item.i_category, item.i_brand, call_center.cc_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [item.i_category, item.i_brand, call_center.cc_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] │ SortExec: expr=[i_category@0 ASC NULLS LAST, i_brand@1 ASC NULLS LAST, cc_name@2 ASC NULLS LAST, d_year@3 ASC NULLS LAST, d_moy@4 ASC NULLS LAST], preserve_partitioning=[true] @@ -5800,7 +5800,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@2)], projection=[i_brand@1, i_category@2, cs_sold_date_sk@3, cs_call_center_sk@4, cs_sales_price@6] │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[i_category@0 as i_category, i_brand@1 as i_brand, cc_name@2 as cc_name, sum(catalog_sales.cs_sales_price)@5 as sum_sales, rank() PARTITION BY [item.i_category, item.i_brand, call_center.cc_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as rn, CAST(rank() PARTITION BY [item.i_category, item.i_brand, call_center.cc_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 AS Decimal128(20, 0)) - Some(1),20,0 as v1_lead.rn - Decimal128(Some(1),20,0)] │ BoundedWindowAggExec: wdw=[rank() PARTITION BY [item.i_category, item.i_brand, call_center.cc_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [item.i_category, item.i_brand, call_center.cc_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] │ SortExec: expr=[i_category@0 ASC NULLS LAST, i_brand@1 ASC NULLS LAST, cc_name@2 ASC NULLS LAST, d_year@3 ASC NULLS LAST, d_moy@4 ASC NULLS LAST], preserve_partitioning=[true] @@ -5823,7 +5823,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@2)], projection=[i_brand@1, i_category@2, cs_sold_date_sk@3, cs_call_center_sk@4, cs_sales_price@6] │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] │ ProjectionExec: expr=[cc_call_center_sk@0 as cc_call_center_sk, cc_name@1 as cc_name, CAST(cc_call_center_sk@0 AS Float64) as CAST(call_center.cc_call_center_sk AS Float64)] @@ -5845,10 +5845,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_item_sk@2], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([cs_item_sk@2], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_call_center_sk, cs_item_sk, cs_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p1] t1:[p2..p3] @@ -5871,10 +5871,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_item_sk@2], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([cs_item_sk@2], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_call_center_sk, cs_item_sk, cs_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 9 ── Tasks: t0:[p0..p1] t1:[p2..p3] @@ -5897,10 +5897,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_item_sk@2], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([cs_item_sk@2], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_call_center_sk, cs_item_sk, cs_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── "#); @@ -5926,7 +5926,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(d_date@2, d_date@0)], projection=[ws_ext_sales_price@0, i_item_id@1] │ CoalescePartitionsExec - │ [Stage 5] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 5] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_week_seq@0, d_week_seq@1)], projection=[d_date@1] │ CoalescePartitionsExec @@ -5945,7 +5945,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(d_date@2, d_date@0)], projection=[cs_ext_sales_price@0, i_item_id@1] │ CoalescePartitionsExec - │ [Stage 11] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 11] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_week_seq@0, d_week_seq@1)], projection=[d_date@1] │ CoalescePartitionsExec @@ -5960,7 +5960,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(d_date@2, d_date@0)], projection=[ss_ext_sales_price@0, i_item_id@1] │ CoalescePartitionsExec - │ [Stage 17] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 17] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_week_seq@0, d_week_seq@1)], projection=[d_date@1] │ CoalescePartitionsExec @@ -5968,41 +5968,41 @@ mod tests { │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date, d_week_seq], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ ProjectionExec: expr=[ws_ext_sales_price@1 as ws_ext_sales_price, i_item_id@2 as i_item_id, d_date@0 as d_date] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ws_sold_date_sk@0)], projection=[d_date@1, ws_ext_sales_price@4, i_item_id@5] │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@2], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@2], 9), input_partitions=3 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ws_sold_date_sk@0], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([ws_sold_date_sk@0], 9), input_partitions=3 │ ProjectionExec: expr=[ws_sold_date_sk@1 as ws_sold_date_sk, ws_ext_sales_price@2 as ws_ext_sales_price, i_item_id@0 as i_item_id] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_item_id@1, ws_sold_date_sk@2, ws_ext_sales_price@4] │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([i_item_sk@0], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([i_item_sk@0], 9), input_partitions=3 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ws_item_sk@1], 6), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ws_item_sk@1], 9), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -6012,41 +6012,41 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date, d_week_seq], file_type=parquet, predicate=d_date@0 = 2000-01-03, pruning_predicate=d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-01-03 AND 2000-01-03 <= d_date_max@1, required_guarantees=[d_date in (2000-01-03)] └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ ProjectionExec: expr=[cs_ext_sales_price@1 as cs_ext_sales_price, i_item_id@2 as i_item_id, d_date@0 as d_date] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, cs_sold_date_sk@0)], projection=[d_date@1, cs_ext_sales_price@4, i_item_id@5] │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@2], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@2], 9), input_partitions=3 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 10 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_sold_date_sk@0], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([cs_sold_date_sk@0], 9), input_partitions=3 │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_ext_sales_price@2 as cs_ext_sales_price, i_item_id@0 as i_item_id] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_item_id@1, cs_sold_date_sk@2, cs_ext_sales_price@4] │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([i_item_sk@0], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([i_item_sk@0], 9), input_partitions=3 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_item_sk@1], 6), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([cs_item_sk@1], 9), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_item_sk, cs_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -6056,41 +6056,41 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date, d_week_seq], file_type=parquet, predicate=d_date@0 = 2000-01-03, pruning_predicate=d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-01-03 AND 2000-01-03 <= d_date_max@1, required_guarantees=[d_date in (2000-01-03)] └────────────────────────────────────────────────── - ┌───── Stage 17 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 17 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ ProjectionExec: expr=[ss_ext_sales_price@1 as ss_ext_sales_price, i_item_id@2 as i_item_id, d_date@0 as d_date] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_date@1, ss_ext_sales_price@4, i_item_id@5] │ [Stage 13] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 16] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 16] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 13 ── Tasks: t0:[p0..p8] t1:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@2], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@2], 9), input_partitions=3 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 16 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_sold_date_sk@0], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([ss_sold_date_sk@0], 9), input_partitions=3 │ ProjectionExec: expr=[ss_sold_date_sk@1 as ss_sold_date_sk, ss_ext_sales_price@2 as ss_ext_sales_price, i_item_id@0 as i_item_id] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_item_id@1, ss_sold_date_sk@2, ss_ext_sales_price@4] │ [Stage 14] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 14 ── Tasks: t0:[p0..p8] t1:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([i_item_sk@0], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([i_item_sk@0], 9), input_partitions=3 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 15 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 15 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_item_sk@1], 6), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_item_sk@1], 9), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 18 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -6125,7 +6125,7 @@ mod tests { │ [Stage 1] => NetworkCoalesceExec: output_partitions=4, input_tasks=2 │ ProjectionExec: expr=[d_week_seq@0 as d_week_seq, ss_store_sk@1 as ss_store_sk, sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END)@2 as sun_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END)@3 as mon_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END)@4 as tue_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END)@5 as wed_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END)@6 as thu_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END)@7 as fri_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)@8 as sat_sales] │ AggregateExec: mode=FinalPartitioned, gby=[d_week_seq@0 as d_week_seq, ss_store_sk@1 as ss_store_sk], aggr=[sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: d_month_seq@0 >= 1212 AND d_month_seq@0 <= 1223, projection=[d_week_seq@1] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 @@ -6141,7 +6141,7 @@ mod tests { │ [Stage 5] => NetworkCoalesceExec: output_partitions=4, input_tasks=2 │ ProjectionExec: expr=[d_week_seq@0 as d_week_seq, ss_store_sk@1 as ss_store_sk, sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END)@2 as sun_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END)@3 as mon_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END)@4 as tue_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END)@5 as wed_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END)@6 as thu_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END)@7 as fri_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)@8 as sat_sales] │ AggregateExec: mode=FinalPartitioned, gby=[d_week_seq@0 as d_week_seq, ss_store_sk@1 as ss_store_sk], aggr=[sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)] - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: d_month_seq@0 >= 1224 AND d_month_seq@0 <= 1235, projection=[d_week_seq@1] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 @@ -6152,7 +6152,7 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, s_store_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 │ RepartitionExec: partitioning=Hash([d_week_seq@0, ss_store_sk@1], 3), input_partitions=3 │ AggregateExec: mode=Partial, gby=[d_week_seq@2 as d_week_seq, ss_store_sk@0 as ss_store_sk], aggr=[sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)] @@ -6160,20 +6160,20 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ss_sold_date_sk@0)], projection=[d_week_seq@1, d_day_name@2, ss_store_sk@5, ss_sales_price@6] │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@3], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@3], 9), input_partitions=3 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_week_seq@1 as d_week_seq, d_day_name@2 as d_day_name, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_week_seq, d_day_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_sold_date_sk@0], 6), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_sold_date_sk@0], 9), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p1] t1:[p2..p3] @@ -6181,7 +6181,7 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 │ RepartitionExec: partitioning=Hash([d_week_seq@0, ss_store_sk@1], 3), input_partitions=3 │ AggregateExec: mode=Partial, gby=[d_week_seq@2 as d_week_seq, ss_store_sk@0 as ss_store_sk], aggr=[sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)] @@ -6189,20 +6189,20 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ss_sold_date_sk@0)], projection=[d_week_seq@1, d_day_name@2, ss_store_sk@5, ss_sales_price@6] │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@3], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@3], 9), input_partitions=3 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_week_seq@1 as d_week_seq, d_day_name@2 as d_day_name, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_week_seq, d_day_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_sold_date_sk@0], 6), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_sold_date_sk@0], 9), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── "#); @@ -6621,14 +6621,11 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, ss_item_sk@1)], projection=[ss_sold_date_sk@1, ss_item_sk@2, ss_customer_sk@3, ss_cdemo_sk@4, ss_hdemo_sk@5, ss_addr_sk@6, ss_store_sk@7, ss_promo_sk@8, ss_wholesale_cost@9, ss_list_price@10, ss_coupon_amt@11] │ CoalescePartitionsExec - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: CAST(sum(catalog_sales.cs_ext_list_price)@1 AS Decimal128(38, 2)) > Some(2),20,0 * sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)@2, projection=[cs_item_sk@0] - │ AggregateExec: mode=FinalPartitioned, gby=[cs_item_sk@0 as cs_item_sk], aggr=[sum(catalog_sales.cs_ext_list_price), sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)] - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 7] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_item_sk@0, ss_item_sk@1), (sr_ticket_number@1, ss_ticket_number@8)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_customer_sk@4, ss_cdemo_sk@5, ss_hdemo_sk@6, ss_addr_sk@7, ss_store_sk@8, ss_promo_sk@9, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13] - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_cdemo_sk@1 as c_current_cdemo_sk, c_current_hdemo_sk@2 as c_current_hdemo_sk, c_current_addr_sk@3 as c_current_addr_sk, c_first_shipto_date_sk@4 as c_first_shipto_date_sk, c_first_sales_date_sk@5 as c_first_sales_date_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_hdemo_sk, c_current_addr_sk, c_first_shipto_date_sk, c_first_sales_date_sk], file_type=parquet @@ -6687,7 +6684,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(promotion.p_promo_sk AS Float64)@1, ss_promo_sk@3)], projection=[ss_item_sk@2, ss_hdemo_sk@3, ss_addr_sk@4, ss_wholesale_cost@6, ss_list_price@7, ss_coupon_amt@8, d_year@9, s_store_name@10, s_zip@11, c_current_hdemo_sk@12, c_current_addr_sk@13, d_year@14, d_year@15] │ CoalescePartitionsExec - │ [Stage 9] => NetworkCoalesceExec: output_partitions=4, input_tasks=2 + │ [Stage 10] => NetworkCoalesceExec: output_partitions=4, input_tasks=2 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_cdemo_sk@10, CAST(cd2.cd_demo_sk AS Float64)@2)], filter=cd_marital_status@1 != cd_marital_status@0, projection=[ss_item_sk@0, ss_hdemo_sk@1, ss_addr_sk@2, ss_promo_sk@3, ss_wholesale_cost@4, ss_list_price@5, ss_coupon_amt@6, d_year@7, s_store_name@8, s_zip@9, c_current_hdemo_sk@11, c_current_addr_sk@12, d_year@13, d_year@14] │ CoalescePartitionsExec @@ -6707,23 +6704,20 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@5)], projection=[s_store_name@1, s_zip@2, ss_item_sk@4, ss_customer_sk@5, ss_cdemo_sk@6, ss_hdemo_sk@7, ss_addr_sk@8, ss_promo_sk@10, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13, d_year@14] │ CoalescePartitionsExec - │ [Stage 10] => NetworkCoalesceExec: output_partitions=4, input_tasks=2 + │ [Stage 11] => NetworkCoalesceExec: output_partitions=4, input_tasks=2 │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_customer_sk@2 as ss_customer_sk, ss_cdemo_sk@3 as ss_cdemo_sk, ss_hdemo_sk@4 as ss_hdemo_sk, ss_addr_sk@5 as ss_addr_sk, ss_store_sk@6 as ss_store_sk, ss_promo_sk@7 as ss_promo_sk, ss_wholesale_cost@8 as ss_wholesale_cost, ss_list_price@9 as ss_list_price, ss_coupon_amt@10 as ss_coupon_amt, d_year@0 as d_year] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_customer_sk@5, ss_cdemo_sk@6, ss_hdemo_sk@7, ss_addr_sk@8, ss_store_sk@9, ss_promo_sk@10, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13] │ CoalescePartitionsExec - │ [Stage 11] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 12] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, ss_item_sk@1)], projection=[ss_sold_date_sk@1, ss_item_sk@2, ss_customer_sk@3, ss_cdemo_sk@4, ss_hdemo_sk@5, ss_addr_sk@6, ss_store_sk@7, ss_promo_sk@8, ss_wholesale_cost@9, ss_list_price@10, ss_coupon_amt@11] │ CoalescePartitionsExec - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: CAST(sum(catalog_sales.cs_ext_list_price)@1 AS Decimal128(38, 2)) > Some(2),20,0 * sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)@2, projection=[cs_item_sk@0] - │ AggregateExec: mode=FinalPartitioned, gby=[cs_item_sk@0 as cs_item_sk], aggr=[sum(catalog_sales.cs_ext_list_price), sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)] - │ [Stage 14] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 16] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_item_sk@0, ss_item_sk@1), (sr_ticket_number@1, ss_ticket_number@8)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_customer_sk@4, ss_cdemo_sk@5, ss_hdemo_sk@6, ss_addr_sk@7, ss_store_sk@8, ss_promo_sk@9, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13] - │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 16] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 17] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 18] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_cdemo_sk@1 as c_current_cdemo_sk, c_current_hdemo_sk@2 as c_current_hdemo_sk, c_current_addr_sk@3 as c_current_addr_sk, c_first_shipto_date_sk@4 as c_first_shipto_date_sk, c_first_sales_date_sk@5 as c_first_sales_date_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_hdemo_sk, c_current_addr_sk, c_first_shipto_date_sk, c_first_sales_date_sk], file_type=parquet @@ -6770,51 +6764,57 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@1 = 1999, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1, required_guarantees=[d_year in (1999)] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_item_sk@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[cs_item_sk@0 as cs_item_sk], aggr=[sum(catalog_sales.cs_ext_list_price), sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)] - │ ProjectionExec: expr=[cs_item_sk@3 as cs_item_sk, cs_ext_list_price@4 as cs_ext_list_price, cr_refunded_cash@0 as cr_refunded_cash, cr_reversed_charge@1 as cr_reversed_charge, cr_store_credit@2 as cr_store_credit] - │ CoalesceBatchesExec: target_batch_size=8192 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(cr_item_sk@0, cs_item_sk@0), (cr_order_number@1, cs_order_number@1)], projection=[cr_refunded_cash@2, cr_reversed_charge@3, cr_store_credit@4, cs_item_sk@5, cs_ext_list_price@7] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ FilterExec: CAST(sum(catalog_sales.cs_ext_list_price)@1 AS Decimal128(38, 2)) > Some(2),20,0 * sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)@2, projection=[cs_item_sk@0] + │ AggregateExec: mode=FinalPartitioned, gby=[cs_item_sk@0 as cs_item_sk], aggr=[sum(catalog_sales.cs_ext_list_price), sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)] + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cr_item_sk@0, cr_order_number@1], 6), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_item_sk, cr_order_number, cr_refunded_cash, cr_reversed_charge, cr_store_credit], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] - │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_item_sk@0, cs_order_number@1], 6), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_item_sk, cs_order_number, cs_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] + │ RepartitionExec: partitioning=Hash([cs_item_sk@0], 6), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[cs_item_sk@0 as cs_item_sk], aggr=[sum(catalog_sales.cs_ext_list_price), sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)] + │ ProjectionExec: expr=[cs_item_sk@3 as cs_item_sk, cs_ext_list_price@4 as cs_ext_list_price, cr_refunded_cash@0 as cr_refunded_cash, cr_reversed_charge@1 as cr_reversed_charge, cr_store_credit@2 as cr_store_credit] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(cr_item_sk@0, cs_item_sk@0), (cr_order_number@1, cs_order_number@1)], projection=[cr_refunded_cash@2, cr_reversed_charge@3, cr_store_credit@4, cs_item_sk@5, cs_ext_list_price@7] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([cr_item_sk@0, cr_order_number@1], 9), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_item_sk, cr_order_number, cr_refunded_cash, cr_reversed_charge, cr_store_credit], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([cs_item_sk@0, cs_order_number@1], 9), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_item_sk, cs_order_number, cs_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([sr_item_sk@0, sr_ticket_number@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([sr_item_sk@0, sr_ticket_number@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_item_sk, sr_ticket_number], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_item_sk@1, ss_ticket_number@8], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_item_sk@1, ss_ticket_number@8], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_cdemo_sk, ss_hdemo_sk, ss_addr_sk, ss_store_sk, ss_promo_sk, ss_ticket_number, ss_wholesale_cost, ss_list_price, ss_coupon_amt], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p1] t1:[p2..p3] + ┌───── Stage 10 ── Tasks: t0:[p0..p1] t1:[p2..p3] │ ProjectionExec: expr=[p_promo_sk@0 as p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)] │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p1] t1:[p2..p3] + ┌───── Stage 11 ── Tasks: t0:[p0..p1] t1:[p2..p3] │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, s_store_name@1 as s_store_name, s_zip@2 as s_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_zip], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p3..p5] │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: d_year@1 = 2000 @@ -6822,38 +6822,44 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@1 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 16 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_item_sk@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[cs_item_sk@0 as cs_item_sk], aggr=[sum(catalog_sales.cs_ext_list_price), sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)] - │ ProjectionExec: expr=[cs_item_sk@3 as cs_item_sk, cs_ext_list_price@4 as cs_ext_list_price, cr_refunded_cash@0 as cr_refunded_cash, cr_reversed_charge@1 as cr_reversed_charge, cr_store_credit@2 as cr_store_credit] - │ CoalesceBatchesExec: target_batch_size=8192 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(cr_item_sk@0, cs_item_sk@0), (cr_order_number@1, cs_order_number@1)], projection=[cr_refunded_cash@2, cr_reversed_charge@3, cr_store_credit@4, cs_item_sk@5, cs_ext_list_price@7] - │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 13] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ FilterExec: CAST(sum(catalog_sales.cs_ext_list_price)@1 AS Decimal128(38, 2)) > Some(2),20,0 * sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)@2, projection=[cs_item_sk@0] + │ AggregateExec: mode=FinalPartitioned, gby=[cs_item_sk@0 as cs_item_sk], aggr=[sum(catalog_sales.cs_ext_list_price), sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)] + │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p5] t1:[p0..p5] - │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cr_item_sk@0, cr_order_number@1], 6), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_item_sk, cr_order_number, cr_refunded_cash, cr_reversed_charge, cr_store_credit], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 15 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_item_sk@0, cs_order_number@1], 6), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_item_sk, cs_order_number, cs_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] + │ RepartitionExec: partitioning=Hash([cs_item_sk@0], 6), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[cs_item_sk@0 as cs_item_sk], aggr=[sum(catalog_sales.cs_ext_list_price), sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)] + │ ProjectionExec: expr=[cs_item_sk@3 as cs_item_sk, cs_ext_list_price@4 as cs_ext_list_price, cr_refunded_cash@0 as cr_refunded_cash, cr_reversed_charge@1 as cr_reversed_charge, cr_store_credit@2 as cr_store_credit] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(cr_item_sk@0, cs_item_sk@0), (cr_order_number@1, cs_order_number@1)], projection=[cr_refunded_cash@2, cr_reversed_charge@3, cr_store_credit@4, cs_item_sk@5, cs_ext_list_price@7] + │ [Stage 13] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 14] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 15 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 13 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([cr_item_sk@0, cr_order_number@1], 9), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_item_sk, cr_order_number, cr_refunded_cash, cr_reversed_charge, cr_store_credit], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 14 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([cs_item_sk@0, cs_order_number@1], 9), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_item_sk, cs_order_number, cs_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 17 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([sr_item_sk@0, sr_ticket_number@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([sr_item_sk@0, sr_ticket_number@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_item_sk, sr_ticket_number], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 18 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_item_sk@1, ss_ticket_number@8], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_item_sk@1, ss_ticket_number@8], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_cdemo_sk, ss_hdemo_sk, ss_addr_sk, ss_store_sk, ss_promo_sk, ss_ticket_number, ss_wholesale_cost, ss_list_price, ss_coupon_amt], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── "); @@ -7373,7 +7379,7 @@ mod tests { │ [Stage 3] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, cs_bill_cdemo_sk@2)], projection=[cs_sold_date_sk@2, cs_ship_date_sk@3, cs_bill_hdemo_sk@5, cs_item_sk@6, cs_promo_sk@7, cs_order_number@8, inv_date_sk@9, w_warehouse_name@10, i_item_desc@11] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ CoalesceBatchesExec: target_batch_size=8192 │ RepartitionExec: partitioning=Hash([cs_bill_cdemo_sk@2], 3), input_partitions=3 │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_ship_date_sk@2 as cs_ship_date_sk, cs_bill_cdemo_sk@3 as cs_bill_cdemo_sk, cs_bill_hdemo_sk@4 as cs_bill_hdemo_sk, cs_item_sk@5 as cs_item_sk, cs_promo_sk@6 as cs_promo_sk, cs_order_number@7 as cs_order_number, inv_date_sk@8 as inv_date_sk, w_warehouse_name@9 as w_warehouse_name, i_item_desc@0 as i_item_desc] @@ -7388,8 +7394,8 @@ mod tests { │ ProjectionExec: expr=[cs_sold_date_sk@2 as cs_sold_date_sk, cs_ship_date_sk@3 as cs_ship_date_sk, cs_bill_cdemo_sk@4 as cs_bill_cdemo_sk, cs_bill_hdemo_sk@5 as cs_bill_hdemo_sk, cs_item_sk@6 as cs_item_sk, cs_promo_sk@7 as cs_promo_sk, cs_order_number@8 as cs_order_number, inv_date_sk@0 as inv_date_sk, inv_warehouse_sk@1 as inv_warehouse_sk] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(inv_item_sk@1, cs_item_sk@4)], filter=inv_quantity_on_hand@1 < cs_quantity@0, projection=[inv_date_sk@0, inv_warehouse_sk@2, cs_sold_date_sk@4, cs_ship_date_sk@5, cs_bill_cdemo_sk@6, cs_bill_hdemo_sk@7, cs_item_sk@8, cs_promo_sk@9, cs_order_number@10] - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_week_seq], file_type=parquet, predicate=DynamicFilter [ empty ] │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] @@ -7418,13 +7424,13 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_buy_potential], file_type=parquet, predicate=hd_buy_potential@1 = >10000, pruning_predicate=hd_buy_potential_null_count@2 != row_count@3 AND hd_buy_potential_min@0 <= >10000 AND >10000 <= hd_buy_potential_max@1, required_guarantees=[hd_buy_potential in (>10000)] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 3), input_partitions=3 + │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 3), input_partitions=2 │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: cd_marital_status@1 = D, projection=[cd_demo_sk@0] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status], file_type=parquet, predicate=cd_marital_status@1 = D, pruning_predicate=cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= D AND D <= cd_marital_status_max@1, required_guarantees=[cd_marital_status in (D)] └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] @@ -7434,16 +7440,16 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_desc], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([inv_item_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([inv_item_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/inventory/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], ...]}, projection=[inv_date_sk, inv_item_sk, inv_warehouse_sk, inv_quantity_on_hand], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_item_sk@4], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([cs_item_sk@4], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_ship_date_sk, cs_bill_cdemo_sk, cs_bill_hdemo_sk, cs_item_sk, cs_promo_sk, cs_order_number, cs_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── "); @@ -7541,7 +7547,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@4, ws_bill_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, ws_sold_date_sk@5, ws_net_paid@7] │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)] │ CoalescePartitionsExec @@ -7561,7 +7567,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@4, ss_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, ss_sold_date_sk@5, ss_net_paid@7] │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[c_customer_id@0 as customer_id, c_first_name@1 as customer_first_name, c_last_name@2 as customer_last_name, sum(store_sales.ss_net_paid)@4 as year_total] │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, d_year@3 as d_year], aggr=[sum(store_sales.ss_net_paid)], ordering_mode=PartiallySorted([3]) │ CoalesceBatchesExec: target_batch_size=8192 @@ -7575,7 +7581,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@4, ss_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, ss_sold_date_sk@5, ss_net_paid@7] │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(web_sales.ws_net_paid)@4 as year_total] │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, d_year@3 as d_year], aggr=[sum(web_sales.ws_net_paid)], ordering_mode=PartiallySorted([3]) │ CoalesceBatchesExec: target_batch_size=8192 @@ -7589,7 +7595,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@4, ws_bill_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, ws_sold_date_sk@5, ws_net_paid@7] │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] @@ -7607,10 +7613,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ws_bill_customer_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ws_bill_customer_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_bill_customer_sk, ws_net_paid], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -7629,10 +7635,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_customer_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_customer_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk, ss_net_paid], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -7651,10 +7657,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_customer_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_customer_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk, ss_net_paid], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -7673,10 +7679,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ws_bill_customer_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ws_bill_customer_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_bill_customer_sk, ws_net_paid], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── "); @@ -7970,7 +7976,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_category@1, ss_sold_date_sk@2, ss_ext_sales_price@4] │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] │ CoalesceBatchesExec: target_batch_size=8192 @@ -7979,12 +7985,12 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_category], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_item_sk@1], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([ss_item_sk@1], 6), input_partitions=2 │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ss_store_sk@2 IS NULL, projection=[ss_sold_date_sk@0, ss_item_sk@1, ss_ext_sales_price@3] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_ext_sales_price], file_type=parquet, predicate=ss_store_sk@2 IS NULL AND DynamicFilter [ empty ] AND DynamicFilter [ empty ], pruning_predicate=ss_store_sk_null_count@0 > 0, required_guarantees=[] └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] @@ -8002,7 +8008,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_category@1, ws_sold_date_sk@2, ws_ext_sales_price@4] │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] │ CoalesceBatchesExec: target_batch_size=8192 @@ -8011,12 +8017,12 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_category], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ws_item_sk@1], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([ws_item_sk@1], 6), input_partitions=2 │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: ws_ship_customer_sk@2 IS NULL, projection=[ws_sold_date_sk@0, ws_item_sk@1, ws_ext_sales_price@3] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_ship_customer_sk, ws_ext_sales_price], file_type=parquet, predicate=ws_ship_customer_sk@2 IS NULL AND DynamicFilter [ empty ] AND DynamicFilter [ empty ], pruning_predicate=ws_ship_customer_sk_null_count@0 > 0, required_guarantees=[] └────────────────────────────────────────────────── ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p0..p5] @@ -8034,7 +8040,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_category@1, cs_sold_date_sk@2, cs_ext_sales_price@4] │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p0..p5] │ CoalesceBatchesExec: target_batch_size=8192 @@ -8043,12 +8049,12 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_category], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_item_sk@1], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([cs_item_sk@1], 6), input_partitions=2 │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: cs_ship_addr_sk@1 IS NULL, projection=[cs_sold_date_sk@0, cs_item_sk@2, cs_ext_sales_price@3] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_ship_addr_sk, cs_item_sk, cs_ext_sales_price], file_type=parquet, predicate=cs_ship_addr_sk@1 IS NULL AND DynamicFilter [ empty ] AND DynamicFilter [ empty ], pruning_predicate=cs_ship_addr_sk_null_count@0 > 0, required_guarantees=[] └────────────────────────────────────────────────── "); @@ -8270,8 +8276,8 @@ mod tests { │ ProjectionExec: expr=[ss_sold_date_sk@1 as ss_sold_date_sk, ss_item_sk@2 as ss_item_sk, ss_customer_sk@3 as ss_customer_sk, ss_quantity@4 as ss_quantity, ss_wholesale_cost@5 as ss_wholesale_cost, ss_sales_price@6 as ss_sales_price, sr_ticket_number@0 as sr_ticket_number] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(sr_ticket_number@1, ss_ticket_number@3), (sr_item_sk@0, ss_item_sk@1)], projection=[sr_ticket_number@1, ss_sold_date_sk@2, ss_item_sk@3, ss_customer_sk@4, ss_quantity@6, ss_wholesale_cost@7, ss_sales_price@8] - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[d_year@0 as ws_sold_year, ws_item_sk@1 as ws_item_sk, ws_bill_customer_sk@2 as ws_customer_sk, sum(web_sales.ws_quantity)@3 as ws_qty, sum(web_sales.ws_wholesale_cost)@4 as ws_wc, sum(web_sales.ws_sales_price)@5 as ws_sp] │ AggregateExec: mode=FinalPartitioned, gby=[d_year@0 as d_year, ws_item_sk@1 as ws_item_sk, ws_bill_customer_sk@2 as ws_bill_customer_sk], aggr=[sum(web_sales.ws_quantity), sum(web_sales.ws_wholesale_cost), sum(web_sales.ws_sales_price)] │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=2 @@ -8287,16 +8293,16 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet, predicate=d_year@1 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([sr_ticket_number@1, sr_item_sk@0], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([sr_ticket_number@1, sr_item_sk@0], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_item_sk, sr_ticket_number], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_ticket_number@3, ss_item_sk@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_ticket_number@3, ss_item_sk@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_ticket_number, ss_quantity, ss_wholesale_cost, ss_sales_price], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] @@ -8307,7 +8313,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ws_sold_date_sk@0)], projection=[d_year@1, ws_item_sk@4, ws_bill_customer_sk@5, ws_quantity@6, ws_wholesale_cost@7, ws_sales_price@8] │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] │ CoalesceBatchesExec: target_batch_size=8192 @@ -8317,7 +8323,7 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ CoalesceBatchesExec: target_batch_size=8192 │ RepartitionExec: partitioning=Hash([ws_sold_date_sk@0], 6), input_partitions=3 │ CoalesceBatchesExec: target_batch_size=8192 @@ -8326,19 +8332,19 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(wr_order_number@1, ws_order_number@3), (wr_item_sk@0, ws_item_sk@1)], projection=[wr_order_number@1, ws_sold_date_sk@2, ws_item_sk@3, ws_bill_customer_sk@4, ws_quantity@6, ws_wholesale_cost@7, ws_sales_price@8] │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([wr_order_number@1, wr_item_sk@0], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([wr_order_number@1, wr_item_sk@0], 9), input_partitions=3 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_returns/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_item_sk, wr_order_number], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ws_order_number@3, ws_item_sk@1], 6), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ws_order_number@3, ws_item_sk@1], 9), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_bill_customer_sk, ws_order_number, ws_quantity, ws_wholesale_cost, ws_sales_price], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 13 ── Tasks: t0:[p0..p2] t1:[p0..p2] @@ -8349,7 +8355,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, cs_sold_date_sk@0)], projection=[d_year@1, cs_bill_customer_sk@4, cs_item_sk@5, cs_quantity@6, cs_wholesale_cost@7, cs_sales_price@8] │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p0..p5] │ CoalesceBatchesExec: target_batch_size=8192 @@ -8359,7 +8365,7 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 12 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ CoalesceBatchesExec: target_batch_size=8192 │ RepartitionExec: partitioning=Hash([cs_sold_date_sk@0], 6), input_partitions=3 │ CoalesceBatchesExec: target_batch_size=8192 @@ -8367,19 +8373,19 @@ mod tests { │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_bill_customer_sk@2 as cs_bill_customer_sk, cs_item_sk@3 as cs_item_sk, cs_quantity@4 as cs_quantity, cs_wholesale_cost@5 as cs_wholesale_cost, cs_sales_price@6 as cs_sales_price, cr_order_number@0 as cr_order_number] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(cr_order_number@1, cs_order_number@3), (cr_item_sk@0, cs_item_sk@2)], projection=[cr_order_number@1, cs_sold_date_sk@2, cs_bill_customer_sk@3, cs_item_sk@4, cs_quantity@6, cs_wholesale_cost@7, cs_sales_price@8] - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 10 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cr_order_number@1, cr_item_sk@0], 6), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([cr_order_number@1, cr_item_sk@0], 9), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_item_sk, cr_order_number], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_order_number@3, cs_item_sk@2], 6), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([cs_order_number@3, cs_item_sk@2], 9), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_bill_customer_sk, cs_item_sk, cs_order_number, cs_quantity, cs_wholesale_cost, cs_sales_price], file_type=parquet └────────────────────────────────────────────────── "); @@ -8488,8 +8494,8 @@ mod tests { │ ProjectionExec: expr=[ss_sold_date_sk@2 as ss_sold_date_sk, ss_item_sk@3 as ss_item_sk, ss_store_sk@4 as ss_store_sk, ss_promo_sk@5 as ss_promo_sk, ss_ext_sales_price@6 as ss_ext_sales_price, ss_net_profit@7 as ss_net_profit, sr_return_amt@0 as sr_return_amt, sr_net_loss@1 as sr_net_loss] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(sr_item_sk@0, ss_item_sk@1), (sr_ticket_number@1, ss_ticket_number@4)], projection=[sr_return_amt@2, sr_net_loss@3, ss_sold_date_sk@4, ss_item_sk@5, ss_store_sk@6, ss_promo_sk@7, ss_ext_sales_price@9, ss_net_profit@10] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[catalog channel as channel, concat(catalog_page, catalog_page_id@0) as id, sales@1 as sales, returns_@2 as returns_, profit@3 as profit] │ ProjectionExec: expr=[cp_catalog_page_id@0 as catalog_page_id, sum(catalog_sales.cs_ext_sales_price)@1 as sales, sum(coalesce(catalog_returns.cr_return_amount,Int64(0)))@2 as returns_, sum(catalog_sales.cs_net_profit - coalesce(catalog_returns.cr_net_loss,Int64(0)))@3 as profit] │ AggregateExec: mode=FinalPartitioned, gby=[cp_catalog_page_id@0 as cp_catalog_page_id], aggr=[sum(catalog_sales.cs_ext_sales_price), sum(coalesce(catalog_returns.cr_return_amount,Int64(0))), sum(catalog_sales.cs_net_profit - coalesce(catalog_returns.cr_net_loss,Int64(0)))] @@ -8515,8 +8521,8 @@ mod tests { │ ProjectionExec: expr=[cs_sold_date_sk@2 as cs_sold_date_sk, cs_catalog_page_sk@3 as cs_catalog_page_sk, cs_item_sk@4 as cs_item_sk, cs_promo_sk@5 as cs_promo_sk, cs_ext_sales_price@6 as cs_ext_sales_price, cs_net_profit@7 as cs_net_profit, cr_return_amount@0 as cr_return_amount, cr_net_loss@1 as cr_net_loss] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(cr_item_sk@0, cs_item_sk@2), (cr_order_number@1, cs_order_number@4)], projection=[cr_return_amount@2, cr_net_loss@3, cs_sold_date_sk@4, cs_catalog_page_sk@5, cs_item_sk@6, cs_promo_sk@7, cs_ext_sales_price@9, cs_net_profit@10] - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=2 │ ProjectionExec: expr=[web channel as channel, concat(web_site, web_site_id@0) as id, sales@1 as sales, returns_@2 as returns_, profit@3 as profit] │ ProjectionExec: expr=[web_site_id@0 as web_site_id, sum(web_sales.ws_ext_sales_price)@1 as sales, sum(coalesce(web_returns.wr_return_amt,Int64(0)))@2 as returns_, sum(web_sales.ws_net_profit - coalesce(web_returns.wr_net_loss,Int64(0)))@3 as profit] @@ -8543,7 +8549,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(wr_item_sk@0, ws_item_sk@1), (wr_order_number@1, ws_order_number@4)], projection=[wr_return_amt@2, wr_net_loss@3, ws_sold_date_sk@4, ws_item_sk@5, ws_web_site_sk@6, ws_promo_sk@7, ws_ext_sales_price@9, ws_net_profit@10] │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 16] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 16] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: i_current_price@1 > Some(5000),4,2, projection=[i_item_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 @@ -8582,16 +8588,16 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-08-23 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-09-22, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([sr_item_sk@0, sr_ticket_number@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([sr_item_sk@0, sr_ticket_number@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_item_sk, sr_ticket_number, sr_return_amt, sr_net_loss], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_item_sk@1, ss_ticket_number@4], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_item_sk@1, ss_ticket_number@4], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_promo_sk, ss_ticket_number, ss_ext_sales_price, ss_net_profit], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -8617,16 +8623,16 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-08-23 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-09-22, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cr_item_sk@0, cr_order_number@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([cr_item_sk@0, cr_order_number@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_item_sk, cr_order_number, cr_return_amount, cr_net_loss], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cs_item_sk@2, cs_order_number@4], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([cs_item_sk@2, cs_order_number@4], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_catalog_page_sk, cs_item_sk, cs_promo_sk, cs_order_number, cs_ext_sales_price, cs_net_profit], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] @@ -8657,10 +8663,10 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_returns/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_item_sk, wr_order_number, wr_return_amt, wr_net_loss], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 16 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ws_item_sk@1, ws_order_number@4], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ws_item_sk@1, ws_order_number@4], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_web_site_sk, ws_promo_sk, ws_order_number, ws_ext_sales_price, ws_net_profit], file_type=parquet └────────────────────────────────────────────────── "); @@ -8771,7 +8777,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(inv_item_sk@1, i_item_sk@0)], projection=[inv_date_sk@0, i_item_sk@2, i_item_id@3, i_item_desc@4, i_current_price@5] │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: i_current_price@3 >= Some(6200),4,2 AND i_current_price@3 <= Some(9200),4,2 AND i_manufact_id@4 IN (SET) ([129, 270, 821, 423]), projection=[i_item_sk@0, i_item_id@1, i_item_desc@2, i_current_price@3] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 @@ -8782,10 +8788,10 @@ mod tests { │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@1 >= 2000-05-25 AND d_date@1 <= 2000-07-24 AND DynamicFilter [ empty ], pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-05-25 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-07-24, required_guarantees=[] │ DataSourceExec: file_groups={3 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:.., /testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:.., /testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:.., /testdata/tpcds/plans_sf1_partitions4/store_sales/part-3.parquet:..]]}, projection=[ss_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: inv_quantity_on_hand@2 >= 100 AND inv_quantity_on_hand@2 <= 500, projection=[inv_date_sk@0, inv_item_sk@1] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/inventory/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], ...]}, projection=[inv_date_sk, inv_item_sk, inv_quantity_on_hand], file_type=parquet, predicate=inv_quantity_on_hand@2 >= 100 AND inv_quantity_on_hand@2 <= 500, pruning_predicate=inv_quantity_on_hand_null_count@1 != row_count@2 AND inv_quantity_on_hand_max@0 >= 100 AND inv_quantity_on_hand_null_count@1 != row_count@2 AND inv_quantity_on_hand_min@3 <= 500, required_guarantees=[] └────────────────────────────────────────────────── "); @@ -8845,7 +8851,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, sr_returned_date_sk@0)], projection=[d_date@1, sr_return_quantity@4, i_item_id@5] │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[i_item_id@0 as item_id, sum(catalog_returns.cr_return_quantity)@1 as cr_item_qty] │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(catalog_returns.cr_return_quantity)] │ CoalesceBatchesExec: target_batch_size=8192 @@ -8864,7 +8870,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(d_date_sk@0, cr_returned_date_sk@0)], projection=[d_date@1, cr_return_quantity@3, i_item_id@4] │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 14] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 14] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ ProjectionExec: expr=[wr_returned_date_sk@1 as wr_returned_date_sk, wr_return_quantity@2 as wr_return_quantity, i_item_id@0 as i_item_id] @@ -8909,26 +8915,26 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 │ RepartitionExec: partitioning=Hash([sr_returned_date_sk@0], 3), input_partitions=3 │ ProjectionExec: expr=[sr_returned_date_sk@1 as sr_returned_date_sk, sr_return_quantity@2 as sr_return_quantity, i_item_id@0 as i_item_id] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, sr_item_sk@1)], projection=[i_item_id@1, sr_returned_date_sk@2, sr_return_quantity@4] │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([i_item_sk@0], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([i_item_sk@0], 9), input_partitions=3 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([sr_item_sk@1], 6), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([sr_item_sk@1], 9), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_returned_date_sk, sr_item_sk, sr_return_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p3..p5] @@ -8945,26 +8951,26 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 14 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 │ RepartitionExec: partitioning=Hash([cr_returned_date_sk@0], 3), input_partitions=3 │ ProjectionExec: expr=[cr_returned_date_sk@1 as cr_returned_date_sk, cr_return_quantity@2 as cr_return_quantity, i_item_id@0 as i_item_id] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(i_item_sk@0, cr_item_sk@1)], projection=[i_item_id@1, cr_returned_date_sk@2, cr_return_quantity@4] │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 13] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 13] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 12 ── Tasks: t0:[p0..p8] t1:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([i_item_sk@0], 6), input_partitions=3 + │ RepartitionExec: partitioning=Hash([i_item_sk@0], 9), input_partitions=3 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p5] t1:[p0..p5] + ┌───── Stage 13 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([cr_item_sk@1], 6), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([cr_item_sk@1], 9), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_returned_date_sk, cr_item_sk, cr_return_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── "); @@ -9065,7 +9071,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(wr_item_sk@0, ws_item_sk@1), (wr_order_number@5, ws_order_number@3)], projection=[wr_refunded_cdemo_sk@1, wr_refunded_addr_sk@2, wr_returning_cdemo_sk@3, wr_reason_sk@4, wr_fee@6, wr_refunded_cash@7, ws_sold_date_sk@8, ws_web_page_sk@10, ws_quantity@12, ws_sales_price@13, ws_net_profit@14] │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, CAST(cd_demo_sk@0 AS Float64) as CAST(cd1.cd_demo_sk AS Float64)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: cd_marital_status@1 = M AND cd_education_status@2 = Advanced Degree OR cd_marital_status@1 = S AND cd_education_status@2 = College OR cd_marital_status@1 = W AND cd_education_status@2 = 2 yr Degree @@ -9100,12 +9106,12 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_returns/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_item_sk, wr_refunded_cdemo_sk, wr_refunded_addr_sk, wr_returning_cdemo_sk, wr_reason_sk, wr_order_number, wr_fee, wr_refunded_cash], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ws_item_sk@1, ws_order_number@3], 3), input_partitions=3 + │ RepartitionExec: partitioning=Hash([ws_item_sk@1, ws_order_number@3], 3), input_partitions=2 │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: (ws_net_profit@6 >= Some(10000),7,2 AND ws_net_profit@6 <= Some(20000),7,2 OR ws_net_profit@6 >= Some(15000),7,2 AND ws_net_profit@6 <= Some(30000),7,2 OR ws_net_profit@6 >= Some(5000),7,2 AND ws_net_profit@6 <= Some(25000),7,2) AND (ws_sales_price@5 >= Some(10000),5,2 AND ws_sales_price@5 <= Some(15000),5,2 OR ws_sales_price@5 >= Some(5000),5,2 AND ws_sales_price@5 <= Some(10000),5,2 OR ws_sales_price@5 >= Some(15000),5,2 AND ws_sales_price@5 <= Some(20000),5,2) - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_web_page_sk, ws_order_number, ws_quantity, ws_sales_price, ws_net_profit], file_type=parquet, predicate=(ws_net_profit@6 >= Some(10000),7,2 AND ws_net_profit@6 <= Some(20000),7,2 OR ws_net_profit@6 >= Some(15000),7,2 AND ws_net_profit@6 <= Some(30000),7,2 OR ws_net_profit@6 >= Some(5000),7,2 AND ws_net_profit@6 <= Some(25000),7,2) AND (ws_sales_price@5 >= Some(10000),5,2 AND ws_sales_price@5 <= Some(15000),5,2 OR ws_sales_price@5 >= Some(5000),5,2 AND ws_sales_price@5 <= Some(10000),5,2 OR ws_sales_price@5 >= Some(15000),5,2 AND ws_sales_price@5 <= Some(20000),5,2) AND DynamicFilter [ empty ] AND DynamicFilter [ empty ], pruning_predicate=(ws_net_profit_null_count@1 != row_count@2 AND ws_net_profit_max@0 >= Some(10000),7,2 AND ws_net_profit_null_count@1 != row_count@2 AND ws_net_profit_min@3 <= Some(20000),7,2 OR ws_net_profit_null_count@1 != row_count@2 AND ws_net_profit_max@0 >= Some(15000),7,2 AND ws_net_profit_null_count@1 != row_count@2 AND ws_net_profit_min@3 <= Some(30000),7,2 OR ws_net_profit_null_count@1 != row_count@2 AND ws_net_profit_max@0 >= Some(5000),7,2 AND ws_net_profit_null_count@1 != row_count@2 AND ws_net_profit_min@3 <= Some(25000),7,2) AND (ws_sales_price_null_count@5 != row_count@2 AND ws_sales_price_max@4 >= Some(10000),5,2 AND ws_sales_price_null_count@5 != row_count@2 AND ws_sales_price_min@6 <= Some(15000),5,2 OR ws_sales_price_null_count@5 != row_count@2 AND ws_sales_price_max@4 >= Some(5000),5,2 AND ws_sales_price_null_count@5 != row_count@2 AND ws_sales_price_min@6 <= Some(10000),5,2 OR ws_sales_price_null_count@5 != row_count@2 AND ws_sales_price_max@4 >= Some(15000),5,2 AND ws_sales_price_null_count@5 != row_count@2 AND ws_sales_price_min@6 <= Some(20000),5,2), required_guarantees=[] └────────────────────────────────────────────────── "); @@ -9867,8 +9873,8 @@ mod tests { │ ProjectionExec: expr=[ss_customer_sk@2 as ss_customer_sk, ss_quantity@3 as ss_quantity, ss_sales_price@4 as ss_sales_price, sr_reason_sk@0 as sr_reason_sk, sr_return_quantity@1 as sr_return_quantity] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(sr_item_sk@0, ss_item_sk@0), (sr_ticket_number@2, ss_ticket_number@2)], projection=[sr_reason_sk@1, sr_return_quantity@3, ss_customer_sk@5, ss_quantity@7, ss_sales_price@8] - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] │ ProjectionExec: expr=[r_reason_sk@0 as r_reason_sk, CAST(r_reason_sk@0 AS Float64) as CAST(reason.r_reason_sk AS Float64)] @@ -9878,16 +9884,16 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/reason/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/reason/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/reason/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/reason/part-3.parquet]]}, projection=[r_reason_sk, r_reason_desc], file_type=parquet, predicate=r_reason_desc@1 = reason 28, pruning_predicate=r_reason_desc_null_count@2 != row_count@3 AND r_reason_desc_min@0 <= reason 28 AND reason 28 <= r_reason_desc_max@1, required_guarantees=[r_reason_desc in (reason 28)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([sr_item_sk@0, sr_ticket_number@2], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([sr_item_sk@0, sr_ticket_number@2], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_item_sk, sr_reason_sk, sr_ticket_number, sr_return_quantity], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ss_item_sk@0, ss_ticket_number@2], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ss_item_sk@0, ss_ticket_number@2], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_item_sk, ss_customer_sk, ss_ticket_number, ss_quantity, ss_sales_price], file_type=parquet └────────────────────────────────────────────────── "); @@ -9992,15 +9998,15 @@ mod tests { │ DataSourceExec: file_groups={3 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:.., /testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:.., /testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:.., /testdata/tpcds/plans_sf1_partitions4/web_sales/part-3.parquet:..]]}, projection=[ws_ship_date_sk, ws_ship_addr_sk, ws_web_site_sk, ws_order_number, ws_ext_ship_cost, ws_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ws_order_number@1, ws_order_number@1)], filter=ws_warehouse_sk@1 != ws_warehouse_sk@0, projection=[ws_order_number@1] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(wr_order_number@0, ws_order_number@0)], projection=[wr_order_number@0] │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=2 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ws_order_number@1, ws_order_number@1)], filter=ws_warehouse_sk@1 != ws_warehouse_sk@0, projection=[ws_order_number@1] - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] │ ProjectionExec: expr=[web_site_sk@0 as web_site_sk, CAST(web_site_sk@0 AS Float64) as CAST(web_site.web_site_sk AS Float64)] @@ -10026,16 +10032,16 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet, predicate=d_date@1 >= 1999-02-01 AND d_date@1 <= 1999-04-02, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 1999-02-01 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 1999-04-02, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ws_order_number@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ws_order_number@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_warehouse_sk, ws_order_number], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ws_order_number@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ws_order_number@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_warehouse_sk, ws_order_number], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] @@ -10045,16 +10051,16 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_returns/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_order_number], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ws_order_number@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ws_order_number@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_warehouse_sk, ws_order_number], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([ws_order_number@1], 3), input_partitions=3 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ws_order_number@1], 3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_warehouse_sk, ws_order_number], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ");