From d9c3152f74299596e927aa221e996175be926903 Mon Sep 17 00:00:00 2001 From: Gabriel Musat Mestre Date: Mon, 26 Jan 2026 15:45:31 +0100 Subject: [PATCH] Coalesce partitions above and below network coalesce --- ...tch_coalescing_below_network_boundaries.rs | 27 +- ...lesce_partitions_below_network_coalesce.rs | 165 ++ .../distributed_physical_optimizer_rule.rs | 216 +- src/distributed_planner/mod.rs | 1 + src/execution_plans/network_coalesce.rs | 7 +- tests/clickbench_plans_test.rs | 377 +-- tests/distributed_aggregation.rs | 24 +- tests/distributed_unions.rs | 97 +- tests/join.rs | 40 +- tests/tpcds_plans_test.rs | 2493 +++++++++-------- tests/tpch_plans_test.rs | 364 +-- 11 files changed, 2094 insertions(+), 1717 deletions(-) create mode 100644 src/distributed_planner/coalesce_partitions_below_network_coalesce.rs diff --git a/src/distributed_planner/batch_coalescing_below_network_boundaries.rs b/src/distributed_planner/batch_coalescing_below_network_boundaries.rs index 9063dad8..af2b2a7d 100644 --- a/src/distributed_planner/batch_coalescing_below_network_boundaries.rs +++ b/src/distributed_planner/batch_coalescing_below_network_boundaries.rs @@ -80,11 +80,12 @@ mod tests { assert_snapshot!(explain, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] - │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[] - │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 + ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] + │ CoalescePartitionsExec + │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[] + │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] t2:[p0..p7] │ RepartitionExec: partitioning=Hash([RainToday@0, WindGustDir@1], 8), input_partitions=4 @@ -108,11 +109,12 @@ mod tests { assert_snapshot!(explain, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] - │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[] - │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 + ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] + │ CoalescePartitionsExec + │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[] + │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] t2:[p0..p7] │ RepartitionExec: partitioning=Hash([RainToday@0, WindGustDir@1], 8), input_partitions=4 @@ -136,12 +138,13 @@ mod tests { assert_snapshot!(explain, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] + ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] │ CoalesceBatchesExec: target_batch_size=101 - │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[] - │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 + │ CoalescePartitionsExec + │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[] + │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] t2:[p0..p7] │ CoalesceBatchesExec: target_batch_size=101 diff --git a/src/distributed_planner/coalesce_partitions_below_network_coalesce.rs b/src/distributed_planner/coalesce_partitions_below_network_coalesce.rs new file mode 100644 index 00000000..a51f7fe4 --- /dev/null +++ b/src/distributed_planner/coalesce_partitions_below_network_coalesce.rs @@ -0,0 +1,165 @@ +use crate::NetworkCoalesceExec; +use crate::common::require_one_child; +use datafusion::common::DataFusionError; +use datafusion::common::tree_node::{Transformed, TreeNode}; +use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; +use datafusion::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; +use std::sync::Arc; + +pub(crate) fn coalesce_partitions_below_network_coalesce( + plan: Arc, +) -> Result, DataFusionError> { + let result = plan.transform_down(|parent| { + let Some(child) = parent.children().pop() else { + return Ok(Transformed::no(parent)); + }; + + let Some(network_coalesce) = child.as_any().downcast_ref::() else { + return Ok(Transformed::no(parent)); + }; + + let network_coalesce_input = require_one_child(network_coalesce.children())?; + + if network_coalesce_input + .output_partitioning() + .partition_count() + == 1 + { + return Ok(Transformed::no(parent)); + } + + if let Some(sort_merge_exec) = parent.as_any().downcast_ref::() { + let child = Arc::clone(child).with_new_children(vec![Arc::new( + SortPreservingMergeExec::new( + sort_merge_exec.expr().clone(), + require_one_child(network_coalesce.children())?, + ), + )])?; + + let parent = parent.with_new_children(vec![child])?; + + return Ok(Transformed::yes(parent)); + } + + if let Some(_coalesce_exec) = parent.as_any().downcast_ref::() { + let child = Arc::clone(child).with_new_children(vec![Arc::new( + CoalescePartitionsExec::new(require_one_child(network_coalesce.children())?), + )])?; + + let parent = parent.with_new_children(vec![child])?; + + return Ok(Transformed::yes(parent)); + } + + Ok(Transformed::no(parent)) + })?; + + Ok(result.data) +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::test_utils::in_memory_channel_resolver::InMemoryWorkerResolver; + use crate::test_utils::parquet::register_parquet_tables; + use crate::{DistributedExt, DistributedPhysicalOptimizerRule}; + use datafusion::execution::SessionStateBuilder; + use datafusion::prelude::SessionContext; + use itertools::Itertools; + + #[tokio::test] + async fn coalesce_partitions() { + let query = r#" + SELECT DISTINCT "RainToday", "WindGustDir" FROM weather + "#; + let plan = sql_to_plan(query).await; + let mut at_least_one_coalesce = false; + // No CoalesceBatchExec is placed before sending data over the network. + plan.transform_down(|plan| { + let Some(network_coalesce) = plan.as_any().downcast_ref::() else { + return Ok(Transformed::no(plan)); + }; + + at_least_one_coalesce = true; + let child = require_one_child(network_coalesce.children())?; + assert!(child.as_any().is::()); + + Ok(Transformed::no(plan)) + }) + .unwrap(); + + assert!(at_least_one_coalesce); + } + + #[tokio::test] + async fn sort_merge_preserving_exec() { + let query = r#" + SELECT DISTINCT "RainToday", "WindGustDir" FROM weather ORDER BY "WindGustDir" DESC + "#; + let plan = sql_to_plan(query).await; + let mut at_least_one_coalesce = false; + // No CoalesceBatchExec is placed before sending data over the network. + plan.transform_down(|plan| { + let Some(network_coalesce) = plan.as_any().downcast_ref::() else { + return Ok(Transformed::no(plan)); + }; + + at_least_one_coalesce = true; + let child = require_one_child(network_coalesce.children())?; + assert!(child.as_any().is::()); + + Ok(Transformed::no(plan)) + }) + .unwrap(); + + assert!(at_least_one_coalesce); + } + + #[tokio::test] + async fn sort_merge_preserving_exec_no_double_inject() { + let query = r#" + SELECT DISTINCT "RainToday", "WindGustDir" FROM weather ORDER BY "WindGustDir" DESC + "#; + let plan = sql_to_plan(query).await; + let plan = coalesce_partitions_below_network_coalesce(plan).unwrap(); + let mut at_least_one_coalesce = false; + // No CoalesceBatchExec is placed before sending data over the network. + plan.transform_down(|plan| { + let Some(network_coalesce) = plan.as_any().downcast_ref::() else { + return Ok(Transformed::no(plan)); + }; + + at_least_one_coalesce = true; + let child = require_one_child(network_coalesce.children())?; + assert!(child.as_any().is::()); + + let grand_child = require_one_child(child.children())?; + assert!(!grand_child.as_any().is::()); + + Ok(Transformed::no(plan)) + }) + .unwrap(); + + assert!(at_least_one_coalesce); + } + + async fn sql_to_plan(query: &str) -> Arc { + let state = SessionStateBuilder::new() + .with_default_features() + .with_physical_optimizer_rule(Arc::new(DistributedPhysicalOptimizerRule)) + .with_distributed_worker_resolver(InMemoryWorkerResolver::new(3)) + .build(); + + let ctx = SessionContext::new_with_state(state); + let mut queries = query.split(";").collect_vec(); + let last_query = queries.pop().unwrap(); + for query in queries { + ctx.sql(query).await.unwrap(); + } + register_parquet_tables(&ctx).await.unwrap(); + let df = ctx.sql(last_query).await.unwrap(); + + df.create_physical_plan().await.unwrap() + } +} diff --git a/src/distributed_planner/distributed_physical_optimizer_rule.rs b/src/distributed_planner/distributed_physical_optimizer_rule.rs index 6fce62a3..0589f0fb 100644 --- a/src/distributed_planner/distributed_physical_optimizer_rule.rs +++ b/src/distributed_planner/distributed_physical_optimizer_rule.rs @@ -1,5 +1,7 @@ +use super::insert_broadcast::insert_broadcast_execs; use crate::common::require_one_child; use crate::distributed_planner::batch_coalescing_below_network_boundaries; +use crate::distributed_planner::coalesce_partitions_below_network_coalesce::coalesce_partitions_below_network_coalesce; use crate::distributed_planner::plan_annotator::{ AnnotatedPlan, PlanOrNetworkBoundary, annotate_plan, }; @@ -17,8 +19,6 @@ use std::ops::AddAssign; use std::sync::Arc; use uuid::Uuid; -use super::insert_broadcast::insert_broadcast_execs; - /// Physical optimizer rule that inspects the plan, places the appropriate network /// boundaries, and breaks it down into stages that can be executed in a distributed manner. /// @@ -63,6 +63,8 @@ impl PhysicalOptimizerRule for DistributedPhysicalOptimizerRule { if stage_id == 1 { return Ok(original); } + + let distributed = coalesce_partitions_below_network_coalesce(distributed)?; let distributed = batch_coalescing_below_network_boundaries(distributed, cfg)?; Ok(Arc::new(DistributedExec::new(distributed))) @@ -231,13 +233,14 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[count(*)@0 as count(*), RainToday@1 as RainToday] │ SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] - │ SortExec: expr=[count(*)@0 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday, count(Int64(1))@1 as count(Int64(1))] - │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 + ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] + │ SortExec: expr=[count(*)@0 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday, count(Int64(1))@1 as count(Int64(1))] + │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] + │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] t2:[p0..p7] │ RepartitionExec: partitioning=Hash([RainToday@0], 8), input_partitions=1 @@ -261,13 +264,14 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[count(*)@0 as count(*), RainToday@1 as RainToday] │ SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] - │ SortExec: expr=[count(*)@0 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday, count(Int64(1))@1 as count(Int64(1))] - │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=2 + ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] + │ SortExec: expr=[count(*)@0 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday, count(Int64(1))@1 as count(Int64(1))] + │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] + │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=2 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] │ RepartitionExec: partitioning=Hash([RainToday@0], 8), input_partitions=2 @@ -364,13 +368,14 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[count(*)@0 as count(*), RainToday@1 as RainToday] │ SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] - │ SortExec: expr=[count(*)@0 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday, count(Int64(1))@1 as count(Int64(1))] - │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 + ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] + │ SortExec: expr=[count(*)@0 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday, count(Int64(1))@1 as count(Int64(1))] + │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] + │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] t2:[p0..p7] │ RepartitionExec: partitioning=Hash([RainToday@0], 8), input_partitions=1 @@ -432,15 +437,16 @@ mod tests { │ CoalescePartitionsExec │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(RainTomorrow@1, RainTomorrow@1)], projection=[MinTemp@0, MaxTemp@2] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 │ ProjectionExec: expr=[avg(weather.MaxTemp)@1 as MaxTemp, RainTomorrow@0 as RainTomorrow] │ AggregateExec: mode=FinalPartitioned, gby=[RainTomorrow@0 as RainTomorrow], aggr=[avg(weather.MaxTemp)] │ [Stage 3] => NetworkShuffleExec: output_partitions=4, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] - │ ProjectionExec: expr=[avg(weather.MinTemp)@1 as MinTemp, RainTomorrow@0 as RainTomorrow] - │ AggregateExec: mode=FinalPartitioned, gby=[RainTomorrow@0 as RainTomorrow], aggr=[avg(weather.MinTemp)] - │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 + ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] + │ CoalescePartitionsExec + │ ProjectionExec: expr=[avg(weather.MinTemp)@1 as MinTemp, RainTomorrow@0 as RainTomorrow] + │ AggregateExec: mode=FinalPartitioned, gby=[RainTomorrow@0 as RainTomorrow], aggr=[avg(weather.MinTemp)] + │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] t2:[p0..p7] │ RepartitionExec: partitioning=Hash([RainTomorrow@0], 8), input_partitions=4 @@ -495,11 +501,12 @@ mod tests { assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] - │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[] - │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 + ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] + │ CoalescePartitionsExec + │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[] + │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] t2:[p0..p7] │ RepartitionExec: partitioning=Hash([RainToday@0, WindGustDir@1], 8), input_partitions=1 @@ -572,19 +579,20 @@ mod tests { assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=24, input_tasks=6 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=6 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] t4:[p16..p19] t5:[p20..p23] - │ DistributedUnionExec: t0:[c0(0/3)] t1:[c0(1/3)] t2:[c0(2/3)] t3:[c1(0/3)] t4:[c1(1/3)] t5:[c1(2/3)] - │ FilterExec: MinTemp@0 > 10 - │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 - │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet, predicate=MinTemp@0 > 10, pruning_predicate=MinTemp_null_count@1 != row_count@2 AND MinTemp_max@0 > 10, required_guarantees=[] - │ ProjectionExec: expr=[MaxTemp@0 as MinTemp, RainToday@1 as RainToday] - │ FilterExec: MaxTemp@0 < 30 + ┌───── Stage 1 ── Tasks: t0:[p0] t1:[p1] t2:[p2] t3:[p3] t4:[p4] t5:[p5] + │ CoalescePartitionsExec + │ DistributedUnionExec: t0:[c0(0/3)] t1:[c0(1/3)] t2:[c0(2/3)] t3:[c1(0/3)] t4:[c1(1/3)] t5:[c1(2/3)] + │ FilterExec: MinTemp@0 > 10 │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet, predicate=MaxTemp@1 < 30, pruning_predicate=MaxTemp_null_count@1 != row_count@2 AND MaxTemp_min@0 < 30, required_guarantees=[] + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet, predicate=MinTemp@0 > 10, pruning_predicate=MinTemp_null_count@1 != row_count@2 AND MinTemp_max@0 > 10, required_guarantees=[] + │ ProjectionExec: expr=[MaxTemp@0 as MinTemp, RainToday@1 as RainToday] + │ FilterExec: MaxTemp@0 < 30 + │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet, predicate=MaxTemp@1 < 30, pruning_predicate=MaxTemp_null_count@1 != row_count@2 AND MaxTemp_min@0 < 30, required_guarantees=[] └────────────────────────────────────────────────── "); } @@ -604,18 +612,19 @@ mod tests { assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=12, input_tasks=3 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] - │ DistributedUnionExec: t0:[c0] t1:[c1(0/2)] t2:[c1(1/2)] - │ FilterExec: MinTemp@0 > 10 - │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet, predicate=MinTemp@0 > 10, pruning_predicate=MinTemp_null_count@1 != row_count@2 AND MinTemp_max@0 > 10, required_guarantees=[] - │ ProjectionExec: expr=[MaxTemp@0 as MinTemp, RainToday@1 as RainToday] - │ FilterExec: MaxTemp@0 < 30 - │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__] t1:[__,__,p0] - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet, predicate=MaxTemp@1 < 30, pruning_predicate=MaxTemp_null_count@1 != row_count@2 AND MaxTemp_min@0 < 30, required_guarantees=[] + ┌───── Stage 1 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ DistributedUnionExec: t0:[c0] t1:[c1(0/2)] t2:[c1(1/2)] + │ FilterExec: MinTemp@0 > 10 + │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet, predicate=MinTemp@0 > 10, pruning_predicate=MinTemp_null_count@1 != row_count@2 AND MinTemp_max@0 > 10, required_guarantees=[] + │ ProjectionExec: expr=[MaxTemp@0 as MinTemp, RainToday@1 as RainToday] + │ FilterExec: MaxTemp@0 < 30 + │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__] t1:[__,__,p0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet, predicate=MaxTemp@1 < 30, pruning_predicate=MaxTemp_null_count@1 != row_count@2 AND MaxTemp_min@0 < 30, required_guarantees=[] └────────────────────────────────────────────────── "); } @@ -637,21 +646,22 @@ mod tests { assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=12, input_tasks=3 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] - │ DistributedUnionExec: t0:[c0] t1:[c1] t2:[c2] - │ FilterExec: MinTemp@0 > 10 - │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet, predicate=MinTemp@0 > 10, pruning_predicate=MinTemp_null_count@1 != row_count@2 AND MinTemp_max@0 > 10, required_guarantees=[] - │ ProjectionExec: expr=[MaxTemp@0 as MinTemp, RainToday@1 as RainToday] - │ FilterExec: MaxTemp@0 < 30 - │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet, predicate=MaxTemp@1 < 30, pruning_predicate=MaxTemp_null_count@1 != row_count@2 AND MaxTemp_min@0 < 30, required_guarantees=[] - │ ProjectionExec: expr=[Temp9am@0 as MinTemp, RainToday@1 as RainToday] - │ FilterExec: Temp9am@0 > 15 + ┌───── Stage 1 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ DistributedUnionExec: t0:[c0] t1:[c1] t2:[c2] + │ FilterExec: MinTemp@0 > 10 │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[Temp9am, RainToday], file_type=parquet, predicate=Temp9am@17 > 15, pruning_predicate=Temp9am_null_count@1 != row_count@2 AND Temp9am_max@0 > 15, required_guarantees=[] + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet, predicate=MinTemp@0 > 10, pruning_predicate=MinTemp_null_count@1 != row_count@2 AND MinTemp_max@0 > 10, required_guarantees=[] + │ ProjectionExec: expr=[MaxTemp@0 as MinTemp, RainToday@1 as RainToday] + │ FilterExec: MaxTemp@0 < 30 + │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet, predicate=MaxTemp@1 < 30, pruning_predicate=MaxTemp_null_count@1 != row_count@2 AND MaxTemp_min@0 < 30, required_guarantees=[] + │ ProjectionExec: expr=[Temp9am@0 as MinTemp, RainToday@1 as RainToday] + │ FilterExec: Temp9am@0 > 15 + │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[Temp9am, RainToday], file_type=parquet, predicate=Temp9am@17 > 15, pruning_predicate=Temp9am_null_count@1 != row_count@2 AND Temp9am_max@0 > 15, required_guarantees=[] └────────────────────────────────────────────────── "); } @@ -677,29 +687,30 @@ mod tests { assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=24, input_tasks=3 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p8..p15] t2:[p16..p23] - │ DistributedUnionExec: t0:[c0, c1] t1:[c2, c3] t2:[c4] - │ FilterExec: MinTemp@0 > 10 - │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet, predicate=MinTemp@0 > 10, pruning_predicate=MinTemp_null_count@1 != row_count@2 AND MinTemp_max@0 > 10, required_guarantees=[] - │ ProjectionExec: expr=[MaxTemp@0 as MinTemp, RainToday@1 as RainToday] - │ FilterExec: MaxTemp@0 < 30 - │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet, predicate=MaxTemp@1 < 30, pruning_predicate=MaxTemp_null_count@1 != row_count@2 AND MaxTemp_min@0 < 30, required_guarantees=[] - │ ProjectionExec: expr=[Temp9am@0 as MinTemp, RainToday@1 as RainToday] - │ FilterExec: Temp9am@0 > 15 - │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[Temp9am, RainToday], file_type=parquet, predicate=Temp9am@17 > 15, pruning_predicate=Temp9am_null_count@1 != row_count@2 AND Temp9am_max@0 > 15, required_guarantees=[] - │ ProjectionExec: expr=[Temp3pm@0 as MinTemp, RainToday@1 as RainToday] - │ FilterExec: Temp3pm@0 < 25 - │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[Temp3pm, RainToday], file_type=parquet, predicate=Temp3pm@18 < 25, pruning_predicate=Temp3pm_null_count@1 != row_count@2 AND Temp3pm_min@0 < 25, required_guarantees=[] - │ ProjectionExec: expr=[Rainfall@0 as MinTemp, RainToday@1 as RainToday] - │ FilterExec: Rainfall@0 > 5 + ┌───── Stage 1 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ DistributedUnionExec: t0:[c0, c1] t1:[c2, c3] t2:[c4] + │ FilterExec: MinTemp@0 > 10 │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[Rainfall, RainToday], file_type=parquet, predicate=Rainfall@2 > 5, pruning_predicate=Rainfall_null_count@1 != row_count@2 AND Rainfall_max@0 > 5, required_guarantees=[] + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet, predicate=MinTemp@0 > 10, pruning_predicate=MinTemp_null_count@1 != row_count@2 AND MinTemp_max@0 > 10, required_guarantees=[] + │ ProjectionExec: expr=[MaxTemp@0 as MinTemp, RainToday@1 as RainToday] + │ FilterExec: MaxTemp@0 < 30 + │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet, predicate=MaxTemp@1 < 30, pruning_predicate=MaxTemp_null_count@1 != row_count@2 AND MaxTemp_min@0 < 30, required_guarantees=[] + │ ProjectionExec: expr=[Temp9am@0 as MinTemp, RainToday@1 as RainToday] + │ FilterExec: Temp9am@0 > 15 + │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[Temp9am, RainToday], file_type=parquet, predicate=Temp9am@17 > 15, pruning_predicate=Temp9am_null_count@1 != row_count@2 AND Temp9am_max@0 > 15, required_guarantees=[] + │ ProjectionExec: expr=[Temp3pm@0 as MinTemp, RainToday@1 as RainToday] + │ FilterExec: Temp3pm@0 < 25 + │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[Temp3pm, RainToday], file_type=parquet, predicate=Temp3pm@18 < 25, pruning_predicate=Temp3pm_null_count@1 != row_count@2 AND Temp3pm_min@0 < 25, required_guarantees=[] + │ ProjectionExec: expr=[Rainfall@0 as MinTemp, RainToday@1 as RainToday] + │ FilterExec: Rainfall@0 > 5 + │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[Rainfall, RainToday], file_type=parquet, predicate=Rainfall@2 > 5, pruning_predicate=Rainfall_null_count@1 != row_count@2 AND Rainfall_max@0 > 5, required_guarantees=[] └────────────────────────────────────────────────── "); } @@ -826,25 +837,26 @@ mod tests { assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ DistributedUnionExec: t0:[c0] t1:[c1] t2:[c2] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(RainToday@1, RainToday@1)], projection=[MinTemp@0, MaxTemp@2] - │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet, predicate=DynamicFilter [ empty ] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(RainToday@1, RainToday@1)], projection=[MinTemp@0, MaxTemp@2] - │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet, predicate=DynamicFilter [ empty ] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(RainToday@1, RainToday@1)], projection=[MinTemp@0, MaxTemp@2] - │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 1 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ DistributedUnionExec: t0:[c0] t1:[c1] t2:[c2] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(RainToday@1, RainToday@1)], projection=[MinTemp@0, MaxTemp@2] + │ CoalescePartitionsExec + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet, predicate=DynamicFilter [ empty ] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(RainToday@1, RainToday@1)], projection=[MinTemp@0, MaxTemp@2] + │ CoalescePartitionsExec + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet, predicate=DynamicFilter [ empty ] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(RainToday@1, RainToday@1)], projection=[MinTemp@0, MaxTemp@2] + │ CoalescePartitionsExec + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── "); } diff --git a/src/distributed_planner/mod.rs b/src/distributed_planner/mod.rs index 2f9b2895..72f99b28 100644 --- a/src/distributed_planner/mod.rs +++ b/src/distributed_planner/mod.rs @@ -1,4 +1,5 @@ mod batch_coalescing_below_network_boundaries; +mod coalesce_partitions_below_network_coalesce; mod distributed_config; mod distributed_physical_optimizer_rule; mod insert_broadcast; diff --git a/src/execution_plans/network_coalesce.rs b/src/execution_plans/network_coalesce.rs index 0a15edc6..c44b2ffe 100644 --- a/src/execution_plans/network_coalesce.rs +++ b/src/execution_plans/network_coalesce.rs @@ -150,8 +150,13 @@ impl ExecutionPlan for NetworkCoalesceExec { self: Arc, children: Vec>, ) -> Result> { + let input = require_one_child(children)?; let mut self_clone = self.as_ref().clone(); - self_clone.input_stage.plan = MaybeEncodedPlan::Decoded(require_one_child(children)?); + + self_clone.properties = scale_partitioning_props(input.properties(), |p| { + p * self_clone.input_stage.tasks.len() + }); + self_clone.input_stage.plan = MaybeEncodedPlan::Decoded(input); Ok(Arc::new(self_clone)) } diff --git a/tests/clickbench_plans_test.rs b/tests/clickbench_plans_test.rs index 08a79d82..f61821f5 100644 --- a/tests/clickbench_plans_test.rs +++ b/tests/clickbench_plans_test.rs @@ -31,14 +31,15 @@ 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=3 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] - │ ProjectionExec: expr=[] - │ 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@40 != 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 1 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] + │ ProjectionExec: expr=[] + │ 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@40 != 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(()) @@ -52,12 +53,13 @@ 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=3 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── 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,__,__,__] 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 + ┌───── Stage 1 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(hits.AdvEngineID), count(Int64(1)), avg(hits.ResolutionWidth)] + │ 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 └────────────────────────────────────────────────── "); Ok(()) @@ -90,12 +92,13 @@ mod tests { │ ProjectionExec: expr=[count(alias1)@0 as count(DISTINCT hits.UserID)] │ AggregateExec: mode=Final, gby=[], aggr=[count(alias1)] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── 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=3 + ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[count(alias1)] + │ AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([alias1@0], 6), input_partitions=2 @@ -115,12 +118,13 @@ mod tests { │ ProjectionExec: expr=[count(alias1)@0 as count(DISTINCT hits.SearchPhrase)] │ AggregateExec: mode=Final, gby=[], aggr=[count(alias1)] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── 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=3 + ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[count(alias1)] + │ AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([alias1@0], 6), input_partitions=2 @@ -147,13 +151,14 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[AdvEngineID@0 as AdvEngineID, count(*)@1 as count(*)] │ SortPreservingMergeExec: [count(Int64(1))@2 DESC] - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── 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 2 ── Tasks: t0:[p0] t1:[p1] + │ 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=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([AdvEngineID@0], 6), input_partitions=2 @@ -199,13 +204,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c@2 DESC], fetch=10 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ 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=3 + ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [c@2 DESC] + │ 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=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([RegionID@0], 6), input_partitions=2 @@ -279,13 +285,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c@1 DESC], fetch=10 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── 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 2 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [c@1 DESC] + │ 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] │ RepartitionExec: partitioning=Hash([SearchPhrase@0], 6), input_partitions=2 @@ -332,13 +339,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c@2 DESC], fetch=10 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── 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 2 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [c@2 DESC] + │ 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] │ RepartitionExec: partitioning=Hash([SearchEngineID@0, SearchPhrase@1], 6), input_partitions=2 @@ -358,13 +366,14 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[UserID@0 as UserID, count(*)@1 as count(*)] │ SortPreservingMergeExec: [count(Int64(1))@2 DESC], fetch=10 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ 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=3 + ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [count(Int64(1))@2 DESC] + │ 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=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([UserID@0], 6), input_partitions=2 @@ -383,13 +392,14 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase, count(*)@2 as count(*)] │ SortPreservingMergeExec: [count(Int64(1))@3 DESC], fetch=10 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ 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=3 + ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [count(Int64(1))@3 DESC] + │ 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=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([UserID@0, SearchPhrase@1], 6), input_partitions=2 @@ -416,13 +426,14 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[UserID@0 as UserID, m@1 as m, SearchPhrase@2 as SearchPhrase, count(*)@3 as count(*)] │ SortPreservingMergeExec: [count(Int64(1))@4 DESC], fetch=10 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ 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=3 + ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [count(Int64(1))@4 DESC] + │ 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=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([UserID@0, date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime))@1, SearchPhrase@2], 6), input_partitions=2 @@ -440,12 +451,13 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ FilterExec: UserID@0 = 435090932899640449 - │ 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@9 = 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)] + ┌───── Stage 1 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ FilterExec: UserID@0 = 435090932899640449 + │ 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@9 = 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)] └────────────────────────────────────────────────── "); Ok(()) @@ -459,14 +471,15 @@ 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=3 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] - │ ProjectionExec: expr=[] - │ FilterExec: CAST(URL@0 AS Utf8View) LIKE %google% - │ 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@13 AS Utf8View) LIKE %google% + ┌───── Stage 1 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] + │ ProjectionExec: expr=[] + │ FilterExec: CAST(URL@0 AS Utf8View) LIKE %google% + │ 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@13 AS Utf8View) LIKE %google% └────────────────────────────────────────────────── "); Ok(()) @@ -494,13 +507,14 @@ 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=3 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── 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] - │ FilterExec: CAST(URL@13 AS Utf8View) LIKE %google% - │ 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 ] + ┌───── Stage 1 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ SortPreservingMergeExec: [EventTime@4 ASC NULLS LAST] + │ SortExec: TopK(fetch=10), expr=[EventTime@4 ASC NULLS LAST], preserve_partitioning=[true] + │ FilterExec: CAST(URL@13 AS Utf8View) LIKE %google% + │ 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 ] └────────────────────────────────────────────────── "); Ok(()) @@ -520,13 +534,14 @@ 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=3 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── 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] - │ 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@39 != 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 ()] + ┌───── Stage 1 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ SortPreservingMergeExec: [SearchPhrase@0 ASC NULLS LAST] + │ SortExec: TopK(fetch=10), expr=[SearchPhrase@0 ASC NULLS LAST], preserve_partitioning=[true] + │ 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@39 != 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 ()] └────────────────────────────────────────────────── "); Ok(()) @@ -539,14 +554,15 @@ 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=3 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── 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] - │ 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=[EventTime, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != 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 ()] + ┌───── Stage 1 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ SortPreservingMergeExec: [EventTime@1 ASC NULLS LAST, SearchPhrase@0 ASC NULLS LAST] + │ 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] + │ 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=[EventTime, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != 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 ()] └────────────────────────────────────────────────── "); Ok(()) @@ -558,14 +574,15 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [l@1 DESC], fetch=25 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, 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] - │ 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 2 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [l@1 DESC] + │ 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] + │ 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] │ RepartitionExec: partitioning=Hash([CounterID@0], 6), input_partitions=2 @@ -584,14 +601,15 @@ mod tests { assert_snapshot!(display, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [l@1 DESC], fetch=25 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, 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)] - │ 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 2 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [l@1 DESC] + │ 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)] + │ 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] │ RepartitionExec: partitioning=Hash([regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))@0], 6), input_partitions=2 @@ -611,12 +629,13 @@ 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=3 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── 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))] - │ 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=[CAST(ResolutionWidth@20 AS Int64) as __common_expr_1], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ 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))] + │ 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=[CAST(ResolutionWidth@20 AS Int64) as __common_expr_1], file_type=parquet └────────────────────────────────────────────────── "); Ok(()) @@ -628,13 +647,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c@2 DESC], fetch=10 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── 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 2 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [c@2 DESC] + │ 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] │ RepartitionExec: partitioning=Hash([SearchEngineID@0, ClientIP@1], 6), input_partitions=2 @@ -669,13 +689,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c@1 DESC], fetch=10 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ 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=3 + ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [c@1 DESC] + │ 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=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([URL@0], 6), input_partitions=2 @@ -693,13 +714,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c@2 DESC], fetch=10 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ 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=3 + ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [c@2 DESC] + │ 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=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([URL@0], 6), input_partitions=2 @@ -717,13 +739,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c@4 DESC], fetch=10 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ 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=3 + ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [c@4 DESC] + │ 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=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([ClientIP@0, hits.ClientIP - Int64(1)@1, hits.ClientIP - Int64(2)@2, hits.ClientIP - Int64(3)@3], 6), input_partitions=2 @@ -741,13 +764,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [pageviews@1 DESC], fetch=10 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── 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 2 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [pageviews@1 DESC] + │ 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] │ RepartitionExec: partitioning=Hash([URL@0], 6), input_partitions=2 @@ -766,13 +790,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [pageviews@1 DESC], fetch=10 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── 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 2 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [pageviews@1 DESC] + │ 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] │ RepartitionExec: partitioning=Hash([Title@0], 6), input_partitions=2 @@ -792,13 +817,14 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ GlobalLimitExec: skip=1000, fetch=10 │ SortPreservingMergeExec: [pageviews@1 DESC], fetch=1010 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── 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 2 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [pageviews@1 DESC] + │ 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] │ RepartitionExec: partitioning=Hash([URL@0], 6), input_partitions=2 @@ -818,13 +844,14 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ GlobalLimitExec: skip=1000, fetch=10 │ SortPreservingMergeExec: [pageviews@5 DESC], fetch=1010 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── 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 2 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [pageviews@5 DESC] + │ 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] │ 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 @@ -844,13 +871,14 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ GlobalLimitExec: skip=100, fetch=10 │ SortPreservingMergeExec: [pageviews@2 DESC], fetch=110 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── 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 2 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [pageviews@2 DESC] + │ 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] │ RepartitionExec: partitioning=Hash([URLHash@0, EventDate@1], 6), input_partitions=2 @@ -870,13 +898,14 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ GlobalLimitExec: skip=10000, fetch=10 │ SortPreservingMergeExec: [pageviews@2 DESC], fetch=10010 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── 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 2 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [pageviews@2 DESC] + │ 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] │ RepartitionExec: partitioning=Hash([WindowClientWidth@0, WindowClientHeight@1], 6), input_partitions=2 diff --git a/tests/distributed_aggregation.rs b/tests/distributed_aggregation.rs index b019209f..6f69ca50 100644 --- a/tests/distributed_aggregation.rs +++ b/tests/distributed_aggregation.rs @@ -52,13 +52,14 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[count(*)@0 as count(*), RainToday@1 as RainToday] │ SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: expr=[count(*)@0 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday, count(Int64(1))@1 as count(Int64(1))] - │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] + │ SortExec: expr=[count(*)@0 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday, count(Int64(1))@1 as count(Int64(1))] + │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], 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] │ RepartitionExec: partitioning=Hash([RainToday@0], 6), input_partitions=1 @@ -133,12 +134,13 @@ mod tests { @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday] - │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] + │ CoalescePartitionsExec + │ ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday] + │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], 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] │ RepartitionExec: partitioning=Hash([RainToday@0], 6), input_partitions=1 diff --git a/tests/distributed_unions.rs b/tests/distributed_unions.rs index 5350a3a4..d73f741d 100644 --- a/tests/distributed_unions.rs +++ b/tests/distributed_unions.rs @@ -40,18 +40,19 @@ mod tests { @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [MinTemp@0 ASC NULLS LAST, RainToday@1 ASC NULLS LAST] - │ [Stage 1] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ DistributedUnionExec: t0:[c0] t1:[c1(0/2)] t2:[c1(1/2)] - │ SortExec: expr=[MinTemp@0 ASC NULLS LAST, RainToday@1 ASC NULLS LAST], preserve_partitioning=[true] - │ FilterExec: MinTemp@0 > 10 - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet, predicate=MinTemp@0 > 10, pruning_predicate=MinTemp_null_count@1 != row_count@2 AND MinTemp_max@0 > 10, required_guarantees=[] - │ SortExec: expr=[MinTemp@0 ASC NULLS LAST, RainToday@1 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[MaxTemp@0 as MinTemp, RainToday@1 as RainToday] - │ FilterExec: MaxTemp@0 < 30 - │ PartitionIsolatorExec: t0:[p0,p1,__] t1:[__,__,p0] - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet, predicate=MaxTemp@1 < 30, pruning_predicate=MaxTemp_null_count@1 != row_count@2 AND MaxTemp_min@0 < 30, required_guarantees=[] + ┌───── Stage 1 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ SortPreservingMergeExec: [MinTemp@0 ASC NULLS LAST, RainToday@1 ASC NULLS LAST] + │ DistributedUnionExec: t0:[c0] t1:[c1(0/2)] t2:[c1(1/2)] + │ SortExec: expr=[MinTemp@0 ASC NULLS LAST, RainToday@1 ASC NULLS LAST], preserve_partitioning=[true] + │ FilterExec: MinTemp@0 > 10 + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet, predicate=MinTemp@0 > 10, pruning_predicate=MinTemp_null_count@1 != row_count@2 AND MinTemp_max@0 > 10, required_guarantees=[] + │ SortExec: expr=[MinTemp@0 ASC NULLS LAST, RainToday@1 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[MaxTemp@0 as MinTemp, RainToday@1 as RainToday] + │ FilterExec: MaxTemp@0 < 30 + │ PartitionIsolatorExec: t0:[p0,p1,__] t1:[__,__,p0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet, predicate=MaxTemp@1 < 30, pruning_predicate=MaxTemp_null_count@1 != row_count@2 AND MaxTemp_min@0 < 30, required_guarantees=[] └────────────────────────────────────────────────── ", ); @@ -90,21 +91,22 @@ mod tests { @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [MinTemp@0 ASC NULLS LAST, RainToday@1 ASC NULLS LAST] - │ [Stage 1] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ DistributedUnionExec: t0:[c0] t1:[c1] t2:[c2] - │ SortExec: expr=[MinTemp@0 ASC NULLS LAST, RainToday@1 ASC NULLS LAST], preserve_partitioning=[true] - │ FilterExec: MinTemp@0 > 20 - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet, predicate=MinTemp@0 > 20, pruning_predicate=MinTemp_null_count@1 != row_count@2 AND MinTemp_max@0 > 20, required_guarantees=[] - │ SortExec: expr=[MinTemp@0 ASC NULLS LAST, RainToday@1 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[MaxTemp@0 as MinTemp, RainToday@1 as RainToday] - │ FilterExec: MaxTemp@0 < 25 - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet, predicate=MaxTemp@1 < 25, pruning_predicate=MaxTemp_null_count@1 != row_count@2 AND MaxTemp_min@0 < 25, required_guarantees=[] - │ SortExec: expr=[MinTemp@0 ASC NULLS LAST, RainToday@1 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[Temp9am@0 as MinTemp, RainToday@1 as RainToday] - │ FilterExec: Temp9am@0 > 15 - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[Temp9am, RainToday], file_type=parquet, predicate=Temp9am@17 > 15, pruning_predicate=Temp9am_null_count@1 != row_count@2 AND Temp9am_max@0 > 15, required_guarantees=[] + ┌───── Stage 1 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ SortPreservingMergeExec: [MinTemp@0 ASC NULLS LAST, RainToday@1 ASC NULLS LAST] + │ DistributedUnionExec: t0:[c0] t1:[c1] t2:[c2] + │ SortExec: expr=[MinTemp@0 ASC NULLS LAST, RainToday@1 ASC NULLS LAST], preserve_partitioning=[true] + │ FilterExec: MinTemp@0 > 20 + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet, predicate=MinTemp@0 > 20, pruning_predicate=MinTemp_null_count@1 != row_count@2 AND MinTemp_max@0 > 20, required_guarantees=[] + │ SortExec: expr=[MinTemp@0 ASC NULLS LAST, RainToday@1 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[MaxTemp@0 as MinTemp, RainToday@1 as RainToday] + │ FilterExec: MaxTemp@0 < 25 + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet, predicate=MaxTemp@1 < 25, pruning_predicate=MaxTemp_null_count@1 != row_count@2 AND MaxTemp_min@0 < 25, required_guarantees=[] + │ SortExec: expr=[MinTemp@0 ASC NULLS LAST, RainToday@1 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[Temp9am@0 as MinTemp, RainToday@1 as RainToday] + │ FilterExec: Temp9am@0 > 15 + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[Temp9am, RainToday], file_type=parquet, predicate=Temp9am@17 > 15, pruning_predicate=Temp9am_null_count@1 != row_count@2 AND Temp9am_max@0 > 15, required_guarantees=[] └────────────────────────────────────────────────── ", ); @@ -147,29 +149,30 @@ mod tests { @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [MinTemp@0 ASC NULLS LAST, RainToday@1 ASC NULLS LAST] - │ [Stage 1] => NetworkCoalesceExec: output_partitions=18, input_tasks=3 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ DistributedUnionExec: t0:[c0, c1] t1:[c2, c3] t2:[c4] - │ SortExec: expr=[MinTemp@0 ASC NULLS LAST, RainToday@1 ASC NULLS LAST], preserve_partitioning=[true] - │ FilterExec: MinTemp@0 > 10 - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet, predicate=MinTemp@0 > 10, pruning_predicate=MinTemp_null_count@1 != row_count@2 AND MinTemp_max@0 > 10, required_guarantees=[] - │ SortExec: expr=[MinTemp@0 ASC NULLS LAST, RainToday@1 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[MaxTemp@0 as MinTemp, RainToday@1 as RainToday] - │ FilterExec: MaxTemp@0 < 30 - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet, predicate=MaxTemp@1 < 30, pruning_predicate=MaxTemp_null_count@1 != row_count@2 AND MaxTemp_min@0 < 30, required_guarantees=[] - │ SortExec: expr=[MinTemp@0 ASC NULLS LAST, RainToday@1 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[Temp9am@0 as MinTemp, RainToday@1 as RainToday] - │ FilterExec: Temp9am@0 > 15 - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[Temp9am, RainToday], file_type=parquet, predicate=Temp9am@17 > 15, pruning_predicate=Temp9am_null_count@1 != row_count@2 AND Temp9am_max@0 > 15, required_guarantees=[] - │ SortExec: expr=[MinTemp@0 ASC NULLS LAST, RainToday@1 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[Temp3pm@0 as MinTemp, RainToday@1 as RainToday] - │ FilterExec: Temp3pm@0 < 25 - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[Temp3pm, RainToday], file_type=parquet, predicate=Temp3pm@18 < 25, pruning_predicate=Temp3pm_null_count@1 != row_count@2 AND Temp3pm_min@0 < 25, required_guarantees=[] - │ SortExec: expr=[MinTemp@0 ASC NULLS LAST, RainToday@1 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[Rainfall@0 as MinTemp, RainToday@1 as RainToday] - │ FilterExec: Rainfall@0 > 5 - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[Rainfall, RainToday], file_type=parquet, predicate=Rainfall@2 > 5, pruning_predicate=Rainfall_null_count@1 != row_count@2 AND Rainfall_max@0 > 5, required_guarantees=[] + ┌───── Stage 1 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ SortPreservingMergeExec: [MinTemp@0 ASC NULLS LAST, RainToday@1 ASC NULLS LAST] + │ DistributedUnionExec: t0:[c0, c1] t1:[c2, c3] t2:[c4] + │ SortExec: expr=[MinTemp@0 ASC NULLS LAST, RainToday@1 ASC NULLS LAST], preserve_partitioning=[true] + │ FilterExec: MinTemp@0 > 10 + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet, predicate=MinTemp@0 > 10, pruning_predicate=MinTemp_null_count@1 != row_count@2 AND MinTemp_max@0 > 10, required_guarantees=[] + │ SortExec: expr=[MinTemp@0 ASC NULLS LAST, RainToday@1 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[MaxTemp@0 as MinTemp, RainToday@1 as RainToday] + │ FilterExec: MaxTemp@0 < 30 + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet, predicate=MaxTemp@1 < 30, pruning_predicate=MaxTemp_null_count@1 != row_count@2 AND MaxTemp_min@0 < 30, required_guarantees=[] + │ SortExec: expr=[MinTemp@0 ASC NULLS LAST, RainToday@1 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[Temp9am@0 as MinTemp, RainToday@1 as RainToday] + │ FilterExec: Temp9am@0 > 15 + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[Temp9am, RainToday], file_type=parquet, predicate=Temp9am@17 > 15, pruning_predicate=Temp9am_null_count@1 != row_count@2 AND Temp9am_max@0 > 15, required_guarantees=[] + │ SortExec: expr=[MinTemp@0 ASC NULLS LAST, RainToday@1 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[Temp3pm@0 as MinTemp, RainToday@1 as RainToday] + │ FilterExec: Temp3pm@0 < 25 + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[Temp3pm, RainToday], file_type=parquet, predicate=Temp3pm@18 < 25, pruning_predicate=Temp3pm_null_count@1 != row_count@2 AND Temp3pm_min@0 < 25, required_guarantees=[] + │ SortExec: expr=[MinTemp@0 ASC NULLS LAST, RainToday@1 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[Rainfall@0 as MinTemp, RainToday@1 as RainToday] + │ FilterExec: Rainfall@0 > 5 + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[Rainfall, RainToday], file_type=parquet, predicate=Rainfall@2 > 5, pruning_predicate=Rainfall_null_count@1 != row_count@2 AND Rainfall_max@0 > 5, required_guarantees=[] └────────────────────────────────────────────────── ", ); diff --git a/tests/join.rs b/tests/join.rs index a4c4938a..72d60f0a 100644 --- a/tests/join.rs +++ b/tests/join.rs @@ -106,19 +106,20 @@ mod tests { // Ensure the distributed plan matches our target plan, registering // hive-style partitioning and avoiding data-shuffling repartitions. assert_snapshot!(&distributed_plan, - @" + @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [f_dkey@0 ASC NULLS LAST, timestamp@1 ASC NULLS LAST] - │ [Stage 1] => NetworkCoalesceExec: output_partitions=4, input_tasks=2 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] - │ ProjectionExec: expr=[f_dkey@5 as f_dkey, timestamp@3 as timestamp, value@4 as value, env@0 as env, service@1 as service, host@2 as host] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(d_dkey@3, f_dkey@2)], projection=[env@0, service@1, host@2, timestamp@4, value@5, f_dkey@6] - │ FilterExec: service@1 = log + ┌───── Stage 1 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [f_dkey@0 ASC NULLS LAST, timestamp@1 ASC NULLS LAST] + │ ProjectionExec: expr=[f_dkey@5 as f_dkey, timestamp@3 as timestamp, value@4 as value, env@0 as env, service@1 as service, host@2 as host] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(d_dkey@3, f_dkey@2)], projection=[env@0, service@1, host@2, timestamp@4, value@5, f_dkey@6] + │ FilterExec: service@1 = log + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/join/parquet/dim/d_dkey=A/data0.parquet], [/testdata/join/parquet/dim/d_dkey=B/data0.parquet], [/testdata/join/parquet/dim/d_dkey=C/data0.parquet], [/testdata/join/parquet/dim/d_dkey=D/data0.parquet]]}, projection=[env, service, host, d_dkey], file_type=parquet, predicate=service@1 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/join/parquet/dim/d_dkey=A/data0.parquet], [/testdata/join/parquet/dim/d_dkey=B/data0.parquet], [/testdata/join/parquet/dim/d_dkey=C/data0.parquet], [/testdata/join/parquet/dim/d_dkey=D/data0.parquet]]}, projection=[env, service, host, d_dkey], file_type=parquet, predicate=service@1 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/join/parquet/fact/f_dkey=A/data0.parquet], [/testdata/join/parquet/fact/f_dkey=B/data0.parquet], [/testdata/join/parquet/fact/f_dkey=C/data0.parquet], [/testdata/join/parquet/fact/f_dkey=D/data0.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] + │ DataSourceExec: file_groups={4 groups: [[/testdata/join/parquet/fact/f_dkey=A/data0.parquet], [/testdata/join/parquet/fact/f_dkey=B/data0.parquet], [/testdata/join/parquet/fact/f_dkey=C/data0.parquet], [/testdata/join/parquet/fact/f_dkey=D/data0.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── "); @@ -186,18 +187,19 @@ mod tests { assert_snapshot!(&distributed_plan, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [f_dkey@0 ASC NULLS LAST, time_bin@1 ASC NULLS LAST] - │ [Stage 1] => NetworkCoalesceExec: output_partitions=4, input_tasks=2 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] - │ ProjectionExec: expr=[f_dkey@0 as f_dkey, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 as time_bin, env@2 as env, max(j.value)@3 as max_bin_value] - │ AggregateExec: mode=SinglePartitioned, gby=[f_dkey@0 as f_dkey, date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }, timestamp@2) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@1 as env], aggr=[max(j.value)], ordering_mode=PartiallySorted([0, 1]) - │ ProjectionExec: expr=[f_dkey@3 as f_dkey, env@0 as env, timestamp@1 as timestamp, value@2 as value] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(d_dkey@1, f_dkey@2)], projection=[env@0, timestamp@2, value@3, f_dkey@4] - │ FilterExec: service@1 = log, projection=[env@0, d_dkey@2] + ┌───── Stage 1 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [f_dkey@0 ASC NULLS LAST, time_bin@1 ASC NULLS LAST] + │ ProjectionExec: expr=[f_dkey@0 as f_dkey, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 as time_bin, env@2 as env, max(j.value)@3 as max_bin_value] + │ AggregateExec: mode=SinglePartitioned, gby=[f_dkey@0 as f_dkey, date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }, timestamp@2) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@1 as env], aggr=[max(j.value)], ordering_mode=PartiallySorted([0, 1]) + │ ProjectionExec: expr=[f_dkey@3 as f_dkey, env@0 as env, timestamp@1 as timestamp, value@2 as value] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(d_dkey@1, f_dkey@2)], projection=[env@0, timestamp@2, value@3, f_dkey@4] + │ FilterExec: service@1 = log, projection=[env@0, d_dkey@2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/join/parquet/dim/d_dkey=A/data0.parquet], [/testdata/join/parquet/dim/d_dkey=B/data0.parquet], [/testdata/join/parquet/dim/d_dkey=C/data0.parquet], [/testdata/join/parquet/dim/d_dkey=D/data0.parquet]]}, projection=[env, service, d_dkey], file_type=parquet, predicate=service@1 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/join/parquet/dim/d_dkey=A/data0.parquet], [/testdata/join/parquet/dim/d_dkey=B/data0.parquet], [/testdata/join/parquet/dim/d_dkey=C/data0.parquet], [/testdata/join/parquet/dim/d_dkey=D/data0.parquet]]}, projection=[env, service, d_dkey], file_type=parquet, predicate=service@1 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/join/parquet/fact/f_dkey=A/data0.parquet], [/testdata/join/parquet/fact/f_dkey=B/data0.parquet], [/testdata/join/parquet/fact/f_dkey=C/data0.parquet], [/testdata/join/parquet/fact/f_dkey=D/data0.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] + │ DataSourceExec: file_groups={4 groups: [[/testdata/join/parquet/fact/f_dkey=A/data0.parquet], [/testdata/join/parquet/fact/f_dkey=B/data0.parquet], [/testdata/join/parquet/fact/f_dkey=C/data0.parquet], [/testdata/join/parquet/fact/f_dkey=D/data0.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── "#); diff --git a/tests/tpcds_plans_test.rs b/tests/tpcds_plans_test.rs index 961396ba..49e510f9 100644 --- a/tests/tpcds_plans_test.rs +++ b/tests/tpcds_plans_test.rs @@ -23,16 +23,17 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c_customer_id@0 ASC NULLS LAST], fetch=100 - │ [Stage 9] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 9] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[c_customer_id@0 ASC NULLS LAST], preserve_partitioning=[true] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ctr_store_sk@0, ctr_store_sk@1)], filter=CAST(ctr_total_return@0 AS Decimal128(30, 15)) > avg(ctr2.ctr_total_return) * Float64(1.2)@1, projection=[c_customer_id@2] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[CAST(CAST(avg(ctr2.ctr_total_return)@1 AS Float64) * 1.2 AS Decimal128(30, 15)) as avg(ctr2.ctr_total_return) * Float64(1.2), ctr_store_sk@0 as ctr_store_sk] - │ AggregateExec: mode=FinalPartitioned, gby=[ctr_store_sk@0 as ctr_store_sk], aggr=[avg(ctr2.ctr_total_return)] - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + ┌───── Stage 9 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [c_customer_id@0 ASC NULLS LAST] + │ SortExec: TopK(fetch=100), expr=[c_customer_id@0 ASC NULLS LAST], preserve_partitioning=[true] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ctr_store_sk@0, ctr_store_sk@1)], filter=CAST(ctr_total_return@0 AS Decimal128(30, 15)) > avg(ctr2.ctr_total_return) * Float64(1.2)@1, projection=[c_customer_id@2] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ ProjectionExec: expr=[CAST(CAST(avg(ctr2.ctr_total_return)@1 AS Float64) * 1.2 AS Decimal128(30, 15)) as avg(ctr2.ctr_total_return) * Float64(1.2), ctr_store_sk@0 as ctr_store_sk] + │ AggregateExec: mode=FinalPartitioned, gby=[ctr_store_sk@0 as ctr_store_sk], aggr=[avg(ctr2.ctr_total_return)] + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=2 └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 @@ -112,21 +113,22 @@ mod tests { assert_snapshot!(display, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [d_week_seq1@0 ASC] - │ [Stage 8] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 8] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: expr=[d_week_seq1@0 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[d_week_seq1@0 as d_week_seq1, round(sun_sales1@1 / sun_sales2@8, 2) as r1, round(mon_sales1@2 / mon_sales2@9, 2) as r2, round(tue_sales1@3 / tue_sales2@10, 2) as r3, round(wed_sales1@4 / wed_sales2@11, 2) as r4, round(thu_sales1@5 / thu_sales2@12, 2) as r5, round(fri_sales1@6 / fri_sales2@13, 2) as r6, round(sat_sales1@7 / sat_sales2@14, 2) as round(y.sat_sales1 / z.sat_sales2,Int64(2))] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(y.d_week_seq1 AS Int64)@8, z.d_week_seq2 - Int64(53)@8)], projection=[d_week_seq1@0, sun_sales1@1, mon_sales1@2, tue_sales1@3, wed_sales1@4, thu_sales1@5, fri_sales1@6, sat_sales1@7, sun_sales2@10, mon_sales2@11, tue_sales2@12, wed_sales2@13, thu_sales2@14, fri_sales2@15, sat_sales2@16] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[d_week_seq@0 as d_week_seq2, sun_sales@1 as sun_sales2, mon_sales@2 as mon_sales2, tue_sales@3 as tue_sales2, wed_sales@4 as wed_sales2, thu_sales@5 as thu_sales2, fri_sales@6 as fri_sales2, sat_sales@7 as sat_sales2, CAST(d_week_seq@0 AS Int64) - 53 as z.d_week_seq2 - Int64(53)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_week_seq@0, d_week_seq@0)], projection=[d_week_seq@1, sun_sales@2, mon_sales@3, tue_sales@4, wed_sales@5, thu_sales@6, fri_sales@7, sat_sales@8] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[d_week_seq@0 as d_week_seq, sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN wscs.sales_price ELSE NULL END)@1 as sun_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN wscs.sales_price ELSE NULL END)@2 as mon_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN wscs.sales_price ELSE NULL END)@3 as tue_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN wscs.sales_price ELSE NULL END)@4 as wed_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN wscs.sales_price ELSE NULL END)@5 as thu_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN wscs.sales_price ELSE NULL END)@6 as fri_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN wscs.sales_price ELSE NULL END)@7 as sat_sales] - │ AggregateExec: mode=FinalPartitioned, gby=[d_week_seq@0 as d_week_seq], aggr=[sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN wscs.sales_price ELSE NULL END)] - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + ┌───── Stage 8 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [d_week_seq1@0 ASC] + │ SortExec: expr=[d_week_seq1@0 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[d_week_seq1@0 as d_week_seq1, round(sun_sales1@1 / sun_sales2@8, 2) as r1, round(mon_sales1@2 / mon_sales2@9, 2) as r2, round(tue_sales1@3 / tue_sales2@10, 2) as r3, round(wed_sales1@4 / wed_sales2@11, 2) as r4, round(thu_sales1@5 / thu_sales2@12, 2) as r5, round(fri_sales1@6 / fri_sales2@13, 2) as r6, round(sat_sales1@7 / sat_sales2@14, 2) as round(y.sat_sales1 / z.sat_sales2,Int64(2))] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(y.d_week_seq1 AS Int64)@8, z.d_week_seq2 - Int64(53)@8)], projection=[d_week_seq1@0, sun_sales1@1, mon_sales1@2, tue_sales1@3, wed_sales1@4, thu_sales1@5, fri_sales1@6, sat_sales1@7, sun_sales2@10, mon_sales2@11, tue_sales2@12, wed_sales2@13, thu_sales2@14, fri_sales2@15, sat_sales2@16] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ ProjectionExec: expr=[d_week_seq@0 as d_week_seq2, sun_sales@1 as sun_sales2, mon_sales@2 as mon_sales2, tue_sales@3 as tue_sales2, wed_sales@4 as wed_sales2, thu_sales@5 as thu_sales2, fri_sales@6 as fri_sales2, sat_sales@7 as sat_sales2, CAST(d_week_seq@0 AS Int64) - 53 as z.d_week_seq2 - Int64(53)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_week_seq@0, d_week_seq@0)], projection=[d_week_seq@1, sun_sales@2, mon_sales@3, tue_sales@4, wed_sales@5, thu_sales@6, fri_sales@7, sat_sales@8] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ ProjectionExec: expr=[d_week_seq@0 as d_week_seq, sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN wscs.sales_price ELSE NULL END)@1 as sun_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN wscs.sales_price ELSE NULL END)@2 as mon_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN wscs.sales_price ELSE NULL END)@3 as tue_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN wscs.sales_price ELSE NULL END)@4 as wed_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN wscs.sales_price ELSE NULL END)@5 as thu_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN wscs.sales_price ELSE NULL END)@6 as fri_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN wscs.sales_price ELSE NULL END)@7 as sat_sales] + │ AggregateExec: mode=FinalPartitioned, gby=[d_week_seq@0 as d_week_seq], aggr=[sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN wscs.sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN wscs.sales_price ELSE NULL END)] + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 @@ -201,13 +203,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [d_year@0 ASC NULLS LAST, sum_agg@3 DESC, brand_id@1 ASC NULLS LAST], fetch=100 - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[d_year@0 ASC NULLS LAST, sum_agg@3 DESC, brand_id@1 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[d_year@0 as d_year, i_brand_id@2 as brand_id, i_brand@1 as brand, sum(store_sales.ss_ext_sales_price)@3 as sum_agg] - │ AggregateExec: mode=FinalPartitioned, gby=[d_year@0 as d_year, i_brand@1 as i_brand, i_brand_id@2 as i_brand_id], aggr=[sum(store_sales.ss_ext_sales_price)] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 4 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [d_year@0 ASC NULLS LAST, sum_agg@3 DESC, brand_id@1 ASC NULLS LAST] + │ SortExec: TopK(fetch=100), expr=[d_year@0 ASC NULLS LAST, sum_agg@3 DESC, brand_id@1 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[d_year@0 as d_year, i_brand_id@2 as brand_id, i_brand@1 as brand, sum(store_sales.ss_ext_sales_price)@3 as sum_agg] + │ AggregateExec: mode=FinalPartitioned, gby=[d_year@0 as d_year, i_brand@1 as i_brand, i_brand_id@2 as i_brand_id], aggr=[sum(store_sales.ss_ext_sales_price)] + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([d_year@0, i_brand@1, i_brand_id@2], 6), input_partitions=3 @@ -245,16 +248,17 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [customer_id@0 ASC, customer_first_name@1 ASC, customer_last_name@2 ASC, customer_preferred_cust_flag@3 ASC], fetch=100 - │ [Stage 24] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 24] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 24 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[customer_id@0 ASC, customer_first_name@1 ASC, customer_last_name@2 ASC, customer_preferred_cust_flag@3 ASC], preserve_partitioning=[true] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)], filter=CASE WHEN year_total@0 > Some(0),24,6 THEN year_total@1 / year_total@0 END > CASE WHEN year_total@2 > Some(0),24,6 THEN year_total@3 / year_total@2 END, projection=[customer_id@1, customer_first_name@2, customer_last_name@3, customer_preferred_cust_flag@4] - │ CoalescePartitionsExec - │ [Stage 20] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ 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]) - │ [Stage 23] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 24 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [customer_id@0 ASC, customer_first_name@1 ASC, customer_last_name@2 ASC, customer_preferred_cust_flag@3 ASC] + │ SortExec: TopK(fetch=100), expr=[customer_id@0 ASC, customer_first_name@1 ASC, customer_last_name@2 ASC, customer_preferred_cust_flag@3 ASC], preserve_partitioning=[true] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)], filter=CASE WHEN year_total@0 > Some(0),24,6 THEN year_total@1 / year_total@0 END > CASE WHEN year_total@2 > Some(0),24,6 THEN year_total@3 / year_total@2 END, projection=[customer_id@1, customer_first_name@2, customer_last_name@3, customer_preferred_cust_flag@4] + │ CoalescePartitionsExec + │ [Stage 20] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ 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]) + │ [Stage 23] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 20 ── Tasks: t0:[p0..p5] t1:[p6..p11] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 @@ -621,26 +625,27 @@ mod tests { │ FilterExec: CAST(i_current_price@1 AS Decimal128(30, 15)) > CAST(1.2 * avg(j.i_current_price)@2 AS Decimal128(30, 15)), projection=[ca_state@0] │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(i_category@2, i_category@1)], projection=[ca_state@0, i_current_price@1, avg(j.i_current_price)@3] │ CoalescePartitionsExec - │ [Stage 7] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ [Stage 7] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[CAST(avg(j.i_current_price)@1 AS Float64) as avg(j.i_current_price), i_category@0 as i_category] │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category], aggr=[avg(j.i_current_price)] │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_month_seq@0, d_month_seq@1)], projection=[ca_state@1, i_current_price@3, i_category@4] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=1 - │ ProjectionExec: expr=[ca_state@2 as ca_state, d_month_seq@3 as d_month_seq, i_current_price@0 as i_current_price, i_category@1 as i_category] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_current_price@1, i_category@2, ca_state@3, d_month_seq@5] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_sold_date_sk@1, CAST(d.d_date_sk AS Float64)@2)], projection=[ca_state@0, ss_item_sk@2, d_month_seq@4] + ┌───── Stage 7 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_month_seq@0, d_month_seq@1)], projection=[ca_state@1, i_current_price@3, i_category@4] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=1 + │ ProjectionExec: expr=[ca_state@2 as ca_state, d_month_seq@3 as d_month_seq, i_current_price@0 as i_current_price, i_category@1 as i_category] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_current_price@1, i_category@2, ca_state@3, d_month_seq@5] │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_month_seq@1 as d_month_seq, CAST(d_date_sk@0 AS Float64) as CAST(d.d_date_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ 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_month_seq], file_type=parquet, predicate=DynamicFilter [ empty ] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_sold_date_sk@1, CAST(d.d_date_sk AS Float64)@2)], projection=[ca_state@0, ss_item_sk@2, d_month_seq@4] + │ CoalescePartitionsExec + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_month_seq@1 as d_month_seq, CAST(d_date_sk@0 AS Float64) as CAST(d.d_date_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ 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_month_seq], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p8] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -699,13 +704,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [i_item_id@0 ASC NULLS LAST], fetch=100 - │ [Stage 7] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 7] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[i_item_id@0 as i_item_id, avg(store_sales.ss_quantity)@1 as agg1, avg(store_sales.ss_list_price)@2 as agg2, avg(store_sales.ss_coupon_amt)@3 as agg3, avg(store_sales.ss_sales_price)@4 as agg4] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[avg(store_sales.ss_quantity), avg(store_sales.ss_list_price), avg(store_sales.ss_coupon_amt), avg(store_sales.ss_sales_price)] - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 7 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [i_item_id@0 ASC NULLS LAST] + │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[i_item_id@0 as i_item_id, avg(store_sales.ss_quantity)@1 as agg1, avg(store_sales.ss_list_price)@2 as agg2, avg(store_sales.ss_coupon_amt)@3 as agg3, avg(store_sales.ss_sales_price)@4 as agg4] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[avg(store_sales.ss_quantity), avg(store_sales.ss_list_price), avg(store_sales.ss_coupon_amt), avg(store_sales.ss_sales_price)] + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([i_item_id@0], 6), input_partitions=3 @@ -857,159 +863,175 @@ mod tests { │ NestedLoopJoinExec: join_type=Left │ NestedLoopJoinExec: join_type=Left │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 │ 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=3 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] │ CoalescePartitionsExec - │ [Stage 3] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 3] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_net_paid)] │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=3, 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=3 + │ [Stage 5] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] │ CoalescePartitionsExec - │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_net_paid)] │ CoalescePartitionsExec - │ [Stage 7] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 7] => NetworkCoalesceExec: output_partitions=3, 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=3 + │ [Stage 8] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] │ CoalescePartitionsExec - │ [Stage 9] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 9] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_net_paid)] │ CoalescePartitionsExec - │ [Stage 10] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 10] => NetworkCoalesceExec: output_partitions=3, 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=3 + │ [Stage 11] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] │ CoalescePartitionsExec - │ [Stage 12] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 12] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_net_paid)] │ CoalescePartitionsExec - │ [Stage 13] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 13] => NetworkCoalesceExec: output_partitions=3, 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=3 + │ [Stage 14] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] │ CoalescePartitionsExec - │ [Stage 15] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 15] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_net_paid)] │ CoalescePartitionsExec - │ [Stage 16] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 16] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] - │ ProjectionExec: expr=[] - │ FilterExec: r_reason_sk@0 = 1 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] + ┌───── Stage 1 ── Tasks: t0:[p0] t1:[p1] + │ CoalescePartitionsExec │ ProjectionExec: expr=[] - │ FilterExec: ss_quantity@0 >= 1 AND ss_quantity@0 <= 20 - │ 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@10 >= 1 AND ss_quantity@10 <= 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=[] + │ FilterExec: r_reason_sk@0 = 1 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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 3 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] - │ FilterExec: ss_quantity@0 >= 1 AND ss_quantity@0 <= 20, projection=[ss_ext_discount_amt@1] - │ 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@10 >= 1 AND ss_quantity@10 <= 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 2 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] + │ ProjectionExec: expr=[] + │ FilterExec: ss_quantity@0 >= 1 AND ss_quantity@0 <= 20 + │ 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@10 >= 1 AND ss_quantity@10 <= 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..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_net_paid)] - │ FilterExec: ss_quantity@0 >= 1 AND ss_quantity@0 <= 20, projection=[ss_net_paid@1] - │ 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@10 >= 1 AND ss_quantity@10 <= 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] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] + │ FilterExec: ss_quantity@0 >= 1 AND ss_quantity@0 <= 20, projection=[ss_ext_discount_amt@1] + │ 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@10 >= 1 AND ss_quantity@10 <= 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..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] - │ ProjectionExec: expr=[] - │ FilterExec: ss_quantity@0 >= 21 AND ss_quantity@0 <= 40 + ┌───── Stage 4 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_net_paid)] + │ FilterExec: ss_quantity@0 >= 1 AND ss_quantity@0 <= 20, projection=[ss_net_paid@1] │ 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@10 >= 21 AND ss_quantity@10 <= 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=[] + │ 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@10 >= 1 AND ss_quantity@10 <= 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 6 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] - │ FilterExec: ss_quantity@0 >= 21 AND ss_quantity@0 <= 40, projection=[ss_ext_discount_amt@1] - │ 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@10 >= 21 AND ss_quantity@10 <= 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 5 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] + │ ProjectionExec: expr=[] + │ FilterExec: ss_quantity@0 >= 21 AND ss_quantity@0 <= 40 + │ 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@10 >= 21 AND ss_quantity@10 <= 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..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_net_paid)] - │ FilterExec: ss_quantity@0 >= 21 AND ss_quantity@0 <= 40, projection=[ss_net_paid@1] - │ 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@10 >= 21 AND ss_quantity@10 <= 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] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] + │ FilterExec: ss_quantity@0 >= 21 AND ss_quantity@0 <= 40, projection=[ss_ext_discount_amt@1] + │ 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@10 >= 21 AND ss_quantity@10 <= 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..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] - │ ProjectionExec: expr=[] - │ FilterExec: ss_quantity@0 >= 41 AND ss_quantity@0 <= 60 + ┌───── Stage 7 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_net_paid)] + │ FilterExec: ss_quantity@0 >= 21 AND ss_quantity@0 <= 40, projection=[ss_net_paid@1] │ 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@10 >= 41 AND ss_quantity@10 <= 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=[] + │ 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@10 >= 21 AND ss_quantity@10 <= 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 9 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] - │ FilterExec: ss_quantity@0 >= 41 AND ss_quantity@0 <= 60, projection=[ss_ext_discount_amt@1] - │ 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@10 >= 41 AND ss_quantity@10 <= 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 8 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] + │ ProjectionExec: expr=[] + │ FilterExec: ss_quantity@0 >= 41 AND ss_quantity@0 <= 60 + │ 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@10 >= 41 AND ss_quantity@10 <= 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..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_net_paid)] - │ FilterExec: ss_quantity@0 >= 41 AND ss_quantity@0 <= 60, projection=[ss_net_paid@1] - │ 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@10 >= 41 AND ss_quantity@10 <= 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] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] + │ FilterExec: ss_quantity@0 >= 41 AND ss_quantity@0 <= 60, projection=[ss_ext_discount_amt@1] + │ 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@10 >= 41 AND ss_quantity@10 <= 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..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] - │ ProjectionExec: expr=[] - │ FilterExec: ss_quantity@0 >= 61 AND ss_quantity@0 <= 80 + ┌───── Stage 10 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_net_paid)] + │ FilterExec: ss_quantity@0 >= 41 AND ss_quantity@0 <= 60, projection=[ss_net_paid@1] │ 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@10 >= 61 AND ss_quantity@10 <= 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=[] + │ 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@10 >= 41 AND ss_quantity@10 <= 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 12 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] - │ FilterExec: ss_quantity@0 >= 61 AND ss_quantity@0 <= 80, projection=[ss_ext_discount_amt@1] - │ 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@10 >= 61 AND ss_quantity@10 <= 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 11 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] + │ ProjectionExec: expr=[] + │ FilterExec: ss_quantity@0 >= 61 AND ss_quantity@0 <= 80 + │ 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@10 >= 61 AND ss_quantity@10 <= 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..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_net_paid)] - │ FilterExec: ss_quantity@0 >= 61 AND ss_quantity@0 <= 80, projection=[ss_net_paid@1] - │ 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@10 >= 61 AND ss_quantity@10 <= 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] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] + │ FilterExec: ss_quantity@0 >= 61 AND ss_quantity@0 <= 80, projection=[ss_ext_discount_amt@1] + │ 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@10 >= 61 AND ss_quantity@10 <= 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..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] - │ ProjectionExec: expr=[] - │ FilterExec: ss_quantity@0 >= 81 AND ss_quantity@0 <= 100 + ┌───── Stage 13 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_net_paid)] + │ FilterExec: ss_quantity@0 >= 61 AND ss_quantity@0 <= 80, projection=[ss_net_paid@1] │ 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@10 >= 81 AND ss_quantity@10 <= 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=[] + │ 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@10 >= 61 AND ss_quantity@10 <= 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 15 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] - │ FilterExec: ss_quantity@0 >= 81 AND ss_quantity@0 <= 100, projection=[ss_ext_discount_amt@1] - │ 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@10 >= 81 AND ss_quantity@10 <= 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 14 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] + │ ProjectionExec: expr=[] + │ FilterExec: ss_quantity@0 >= 81 AND ss_quantity@0 <= 100 + │ 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@10 >= 81 AND ss_quantity@10 <= 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..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_net_paid)] - │ FilterExec: ss_quantity@0 >= 81 AND ss_quantity@0 <= 100, projection=[ss_net_paid@1] - │ 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@10 >= 81 AND ss_quantity@10 <= 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] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_ext_discount_amt)] + │ FilterExec: ss_quantity@0 >= 81 AND ss_quantity@0 <= 100, projection=[ss_ext_discount_amt@1] + │ 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@10 >= 81 AND ss_quantity@10 <= 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] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_net_paid)] + │ FilterExec: ss_quantity@0 >= 81 AND ss_quantity@0 <= 100, projection=[ss_net_paid@1] + │ 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@10 >= 81 AND ss_quantity@10 <= 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=[] └────────────────────────────────────────────────── "); Ok(()) @@ -1120,16 +1142,17 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [customer_id@0 ASC, customer_first_name@1 ASC, customer_last_name@2 ASC, customer_preferred_cust_flag@3 ASC], fetch=100 - │ [Stage 16] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 16] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[customer_id@0 ASC, customer_first_name@1 ASC, customer_last_name@2 ASC, customer_preferred_cust_flag@3 ASC], preserve_partitioning=[true] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)], filter=CASE WHEN year_total@2 > Some(0),18,2 THEN CAST(year_total@3 AS Float64) / CAST(year_total@2 AS Float64) ELSE 0 END > CASE WHEN year_total@0 > Some(0),18,2 THEN CAST(year_total@1 AS Float64) / CAST(year_total@0 AS Float64) ELSE 0 END, projection=[customer_id@2, customer_first_name@3, customer_last_name@4, customer_preferred_cust_flag@5] - │ CoalescePartitionsExec - │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ 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]) - │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 16 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [customer_id@0 ASC, customer_first_name@1 ASC, customer_last_name@2 ASC, customer_preferred_cust_flag@3 ASC] + │ SortExec: TopK(fetch=100), expr=[customer_id@0 ASC, customer_first_name@1 ASC, customer_last_name@2 ASC, customer_preferred_cust_flag@3 ASC], preserve_partitioning=[true] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)], filter=CASE WHEN year_total@2 > Some(0),18,2 THEN CAST(year_total@3 AS Float64) / CAST(year_total@2 AS Float64) ELSE 0 END > CASE WHEN year_total@0 > Some(0),18,2 THEN CAST(year_total@1 AS Float64) / CAST(year_total@0 AS Float64) ELSE 0 END, projection=[customer_id@2, customer_first_name@3, customer_last_name@4, customer_preferred_cust_flag@5] + │ CoalescePartitionsExec + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ 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]) + │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 12 ── Tasks: t0:[p0..p5] t1:[p6..p11] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 @@ -1331,18 +1354,19 @@ mod tests { │ ProjectionExec: expr=[avg(store_sales.ss_quantity)@0 as avg1, avg(store_sales.ss_ext_sales_price)@1 as avg2, avg(store_sales.ss_ext_wholesale_cost)@2 as avg3, sum(store_sales.ss_ext_wholesale_cost)@3 as sum(store_sales.ss_ext_wholesale_cost)] │ AggregateExec: mode=Final, gby=[], aggr=[avg(store_sales.ss_quantity), avg(store_sales.ss_ext_sales_price), avg(store_sales.ss_ext_wholesale_cost), sum(store_sales.ss_ext_wholesale_cost)] │ CoalescePartitionsExec - │ [Stage 6] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_quantity), avg(store_sales.ss_ext_sales_price), avg(store_sales.ss_ext_wholesale_cost), sum(store_sales.ss_ext_wholesale_cost)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@1)], projection=[ss_quantity@1, ss_ext_sales_price@2, ss_ext_wholesale_cost@3] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ 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)] - │ FilterExec: d_year@1 = 2001, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ 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@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] + ┌───── Stage 6 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_quantity), avg(store_sales.ss_ext_sales_price), avg(store_sales.ss_ext_wholesale_cost), sum(store_sales.ss_ext_wholesale_cost)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@1)], projection=[ss_quantity@1, ss_ext_sales_price@2, ss_ext_wholesale_cost@3] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ 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)] + │ FilterExec: d_year@1 = 2001, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ 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@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -1412,13 +1436,13 @@ mod tests { │ ProjectionExec: expr=[avg(sq2.quantity * sq2.list_price)@0 as average_sales] │ AggregateExec: mode=Final, gby=[], aggr=[avg(sq2.quantity * sq2.list_price)] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=4, input_tasks=4 │ AggregateExec: mode=FinalPartitioned, gby=[i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id], aggr=[sum(store_sales.ss_quantity * store_sales.ss_list_price), count(Int64(1))] │ RepartitionExec: partitioning=Hash([i_brand_id@0, i_class_id@1, i_category_id@2], 3), input_partitions=3 │ AggregateExec: mode=Partial, gby=[i_brand_id@2 as i_brand_id, i_class_id@3 as i_class_id, i_category_id@4 as i_category_id], aggr=[sum(store_sales.ss_quantity * store_sales.ss_list_price), count(Int64(1))] │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(ss_item_sk@0, ss_item_sk@0)], projection=[ss_quantity@1, ss_list_price@2, i_brand_id@3, i_class_id@4, i_category_id@5] │ CoalescePartitionsExec - │ [Stage 5] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 5] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[i_item_sk@0 as ss_item_sk] │ 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] │ CoalescePartitionsExec @@ -1439,13 +1463,13 @@ mod tests { │ ProjectionExec: expr=[avg(sq2.quantity * sq2.list_price)@0 as average_sales] │ AggregateExec: mode=Final, gby=[], aggr=[avg(sq2.quantity * sq2.list_price)] │ CoalescePartitionsExec - │ [Stage 16] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + │ [Stage 16] => NetworkCoalesceExec: output_partitions=4, input_tasks=4 │ AggregateExec: mode=FinalPartitioned, gby=[i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id], aggr=[sum(catalog_sales.cs_quantity * catalog_sales.cs_list_price), count(Int64(1))] │ RepartitionExec: partitioning=Hash([i_brand_id@0, i_class_id@1, i_category_id@2], 3), input_partitions=3 │ AggregateExec: mode=Partial, gby=[i_brand_id@2 as i_brand_id, i_class_id@3 as i_class_id, i_category_id@4 as i_category_id], aggr=[sum(catalog_sales.cs_quantity * catalog_sales.cs_list_price), count(Int64(1))] │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(cs_item_sk@0, ss_item_sk@0)], projection=[cs_quantity@1, cs_list_price@2, i_brand_id@3, i_class_id@4, i_category_id@5] │ CoalescePartitionsExec - │ [Stage 19] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 19] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[i_item_sk@0 as ss_item_sk] │ 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] │ CoalescePartitionsExec @@ -1466,13 +1490,13 @@ mod tests { │ ProjectionExec: expr=[avg(sq2.quantity * sq2.list_price)@0 as average_sales] │ AggregateExec: mode=Final, gby=[], aggr=[avg(sq2.quantity * sq2.list_price)] │ CoalescePartitionsExec - │ [Stage 30] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + │ [Stage 30] => NetworkCoalesceExec: output_partitions=4, input_tasks=4 │ AggregateExec: mode=FinalPartitioned, gby=[i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id], aggr=[sum(web_sales.ws_quantity * web_sales.ws_list_price), count(Int64(1))] │ RepartitionExec: partitioning=Hash([i_brand_id@0, i_class_id@1, i_category_id@2], 3), input_partitions=3 │ AggregateExec: mode=Partial, gby=[i_brand_id@2 as i_brand_id, i_class_id@3 as i_class_id, i_category_id@4 as i_category_id], aggr=[sum(web_sales.ws_quantity * web_sales.ws_list_price), count(Int64(1))] │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(ws_item_sk@0, ss_item_sk@0)], projection=[ws_quantity@1, ws_list_price@2, i_brand_id@3, i_class_id@4, i_category_id@5] │ CoalescePartitionsExec - │ [Stage 33] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 33] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[i_item_sk@0 as ss_item_sk] │ 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] │ CoalescePartitionsExec @@ -1489,33 +1513,34 @@ mod tests { │ 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 ] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] - │ AggregateExec: mode=Partial, gby=[], aggr=[avg(sq2.quantity * sq2.list_price)] - │ DistributedUnionExec: t0:[c0] t1:[c1] t2:[c2(0/2)] t3:[c2(1/2)] - │ ProjectionExec: expr=[ss_quantity@0 as quantity, ss_list_price@1 as list_price] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_quantity@3, ss_list_price@4] - │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 - │ 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)] - │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] - │ 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_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, 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_sold_date_sk, ss_quantity, ss_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] - │ ProjectionExec: expr=[cs_quantity@0 as quantity, cs_list_price@1 as list_price] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_quantity@3, cs_list_price@4] - │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 - │ 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)] - │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] - │ 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_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, 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_sold_date_sk, cs_quantity, cs_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] - │ ProjectionExec: expr=[ws_quantity@0 as quantity, ws_list_price@1 as list_price] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_quantity@3, ws_list_price@4] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] - │ 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_quantity, ws_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] t2:[p2] t3:[p3] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(sq2.quantity * sq2.list_price)] + │ DistributedUnionExec: t0:[c0] t1:[c1] t2:[c2(0/2)] t3:[c2(1/2)] + │ ProjectionExec: expr=[ss_quantity@0 as quantity, ss_list_price@1 as list_price] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_quantity@3, ss_list_price@4] + │ CoalescePartitionsExec + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ 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)] + │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] + │ 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_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, 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_sold_date_sk, ss_quantity, ss_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] + │ ProjectionExec: expr=[cs_quantity@0 as quantity, cs_list_price@1 as list_price] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_quantity@3, cs_list_price@4] + │ CoalescePartitionsExec + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ 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)] + │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] + │ 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_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, 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_sold_date_sk, cs_quantity, cs_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] + │ ProjectionExec: expr=[ws_quantity@0 as quantity, ws_list_price@1 as list_price] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_quantity@3, ws_list_price@4] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ 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_quantity, ws_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 @@ -1525,16 +1550,17 @@ 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@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_quantity@4, ss_list_price@5, i_brand_id@6, i_class_id@7, i_category_id@8] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ss_sold_date_sk@3 as ss_sold_date_sk, ss_item_sk@4 as ss_item_sk, ss_quantity@5 as ss_quantity, ss_list_price@6 as ss_list_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id] - │ HashJoinExec: mode=CollectLeft, 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] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=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 5 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_quantity@4, ss_list_price@5, i_brand_id@6, i_class_id@7, i_category_id@8] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ ProjectionExec: expr=[ss_sold_date_sk@3 as ss_sold_date_sk, ss_item_sk@4 as ss_item_sk, ss_quantity@5 as ss_quantity, ss_list_price@6 as ss_list_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id] + │ HashJoinExec: mode=CollectLeft, 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] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=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 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -1626,33 +1652,34 @@ 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:[p3..p5] t2:[p6..p8] t3:[p9..p11] - │ AggregateExec: mode=Partial, gby=[], aggr=[avg(sq2.quantity * sq2.list_price)] - │ DistributedUnionExec: t0:[c0] t1:[c1] t2:[c2(0/2)] t3:[c2(1/2)] - │ ProjectionExec: expr=[ss_quantity@0 as quantity, ss_list_price@1 as list_price] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_quantity@3, ss_list_price@4] - │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 - │ 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)] - │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] - │ 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_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, 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_sold_date_sk, ss_quantity, ss_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] - │ ProjectionExec: expr=[cs_quantity@0 as quantity, cs_list_price@1 as list_price] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_quantity@3, cs_list_price@4] - │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 - │ 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)] - │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] - │ 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_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, 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_sold_date_sk, cs_quantity, cs_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] - │ ProjectionExec: expr=[ws_quantity@0 as quantity, ws_list_price@1 as list_price] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_quantity@3, ws_list_price@4] - │ CoalescePartitionsExec - │ [Stage 15] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] - │ 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_quantity, ws_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 16 ── Tasks: t0:[p0] t1:[p1] t2:[p2] t3:[p3] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(sq2.quantity * sq2.list_price)] + │ DistributedUnionExec: t0:[c0] t1:[c1] t2:[c2(0/2)] t3:[c2(1/2)] + │ ProjectionExec: expr=[ss_quantity@0 as quantity, ss_list_price@1 as list_price] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_quantity@3, ss_list_price@4] + │ CoalescePartitionsExec + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ 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)] + │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] + │ 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_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, 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_sold_date_sk, ss_quantity, ss_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] + │ ProjectionExec: expr=[cs_quantity@0 as quantity, cs_list_price@1 as list_price] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_quantity@3, cs_list_price@4] + │ CoalescePartitionsExec + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ 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)] + │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] + │ 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_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, 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_sold_date_sk, cs_quantity, cs_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] + │ ProjectionExec: expr=[ws_quantity@0 as quantity, ws_list_price@1 as list_price] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_quantity@3, ws_list_price@4] + │ CoalescePartitionsExec + │ [Stage 15] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ 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_quantity, ws_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 15 ── Tasks: t0:[p0..p5] t1:[p6..p11] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 @@ -1662,16 +1689,17 @@ 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@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 19 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_item_sk@3, cs_quantity@4, cs_list_price@5, i_brand_id@6, i_class_id@7, i_category_id@8] - │ CoalescePartitionsExec - │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[cs_sold_date_sk@3 as cs_sold_date_sk, cs_item_sk@4 as cs_item_sk, cs_quantity@5 as cs_quantity, cs_list_price@6 as cs_list_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id] - │ HashJoinExec: mode=CollectLeft, 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] - │ CoalescePartitionsExec - │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=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 19 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_item_sk@3, cs_quantity@4, cs_list_price@5, i_brand_id@6, i_class_id@7, i_category_id@8] + │ CoalescePartitionsExec + │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ ProjectionExec: expr=[cs_sold_date_sk@3 as cs_sold_date_sk, cs_item_sk@4 as cs_item_sk, cs_quantity@5 as cs_quantity, cs_list_price@6 as cs_list_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id] + │ HashJoinExec: mode=CollectLeft, 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] + │ CoalescePartitionsExec + │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=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 17 ── Tasks: t0:[p0..p8] t1:[p9..p17] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -1763,33 +1791,34 @@ 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 30 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] - │ AggregateExec: mode=Partial, gby=[], aggr=[avg(sq2.quantity * sq2.list_price)] - │ DistributedUnionExec: t0:[c0] t1:[c1] t2:[c2(0/2)] t3:[c2(1/2)] - │ ProjectionExec: expr=[ss_quantity@0 as quantity, ss_list_price@1 as list_price] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_quantity@3, ss_list_price@4] - │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 - │ 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)] - │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] - │ 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_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, 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_sold_date_sk, ss_quantity, ss_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] - │ ProjectionExec: expr=[cs_quantity@0 as quantity, cs_list_price@1 as list_price] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_quantity@3, cs_list_price@4] - │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 - │ 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)] - │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] - │ 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_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, 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_sold_date_sk, cs_quantity, cs_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] - │ ProjectionExec: expr=[ws_quantity@0 as quantity, ws_list_price@1 as list_price] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_quantity@3, ws_list_price@4] - │ CoalescePartitionsExec - │ [Stage 29] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] - │ 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_quantity, ws_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 30 ── Tasks: t0:[p0] t1:[p1] t2:[p2] t3:[p3] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(sq2.quantity * sq2.list_price)] + │ DistributedUnionExec: t0:[c0] t1:[c1] t2:[c2(0/2)] t3:[c2(1/2)] + │ ProjectionExec: expr=[ss_quantity@0 as quantity, ss_list_price@1 as list_price] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_quantity@3, ss_list_price@4] + │ CoalescePartitionsExec + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ 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)] + │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] + │ 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_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, 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_sold_date_sk, ss_quantity, ss_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] + │ ProjectionExec: expr=[cs_quantity@0 as quantity, cs_list_price@1 as list_price] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_quantity@3, cs_list_price@4] + │ CoalescePartitionsExec + │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ 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)] + │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] + │ 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_year], file_type=parquet, predicate=d_year@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, 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_sold_date_sk, cs_quantity, cs_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] + │ ProjectionExec: expr=[ws_quantity@0 as quantity, ws_list_price@1 as list_price] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_quantity@3, ws_list_price@4] + │ CoalescePartitionsExec + │ [Stage 29] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ 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_quantity, ws_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 29 ── Tasks: t0:[p0..p5] t1:[p6..p11] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 @@ -1799,16 +1828,17 @@ 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@6 >= 1999 AND d_year@6 <= 2001, pruning_predicate=d_year_null_count@1 != row_count@2 AND d_year_max@0 >= 1999 AND d_year_null_count@1 != row_count@2 AND d_year_min@3 <= 2001, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 33 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_item_sk@3, ws_quantity@4, ws_list_price@5, i_brand_id@6, i_class_id@7, i_category_id@8] - │ CoalescePartitionsExec - │ [Stage 31] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ws_sold_date_sk@3 as ws_sold_date_sk, ws_item_sk@4 as ws_item_sk, ws_quantity@5 as ws_quantity, ws_list_price@6 as ws_list_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id] - │ HashJoinExec: mode=CollectLeft, 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] - │ CoalescePartitionsExec - │ [Stage 32] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=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 33 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_item_sk@3, ws_quantity@4, ws_list_price@5, i_brand_id@6, i_class_id@7, i_category_id@8] + │ CoalescePartitionsExec + │ [Stage 31] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ ProjectionExec: expr=[ws_sold_date_sk@3 as ws_sold_date_sk, ws_item_sk@4 as ws_item_sk, ws_quantity@5 as ws_quantity, ws_list_price@6 as ws_list_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id] + │ HashJoinExec: mode=CollectLeft, 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] + │ CoalescePartitionsExec + │ [Stage 32] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=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 31 ── Tasks: t0:[p0..p8] t1:[p9..p17] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -1909,12 +1939,13 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [ca_zip@0 ASC], fetch=100 - │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[ca_zip@0 ASC], preserve_partitioning=[true] - │ AggregateExec: mode=FinalPartitioned, gby=[ca_zip@0 as ca_zip], aggr=[sum(catalog_sales.cs_sales_price)] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 6 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [ca_zip@0 ASC] + │ SortExec: TopK(fetch=100), expr=[ca_zip@0 ASC], preserve_partitioning=[true] + │ AggregateExec: mode=FinalPartitioned, gby=[ca_zip@0 as ca_zip], aggr=[sum(catalog_sales.cs_sales_price)] + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([ca_zip@0], 6), input_partitions=3 @@ -1976,22 +2007,23 @@ mod tests { │ CoalescePartitionsExec │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(cs_order_number@1, cs_order_number@1)], filter=cs_warehouse_sk@0 != cs_warehouse_sk@1, projection=[cs_order_number@1, cs_ext_ship_cost@2, cs_net_profit@3] │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ 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_warehouse_sk, cs_order_number], file_type=parquet │ DataSourceExec: file_groups={3 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:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-3.parquet:..]]}, projection=[cr_order_number], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(call_center.cc_call_center_sk AS Float64)@1, cs_call_center_sk@0)], projection=[cs_warehouse_sk@3, cs_order_number@4, cs_ext_ship_cost@5, cs_net_profit@6] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer_address.ca_address_sk AS Float64)@1, cs_ship_addr_sk@0)], projection=[cs_call_center_sk@3, cs_warehouse_sk@4, cs_order_number@5, cs_ext_ship_cost@6, cs_net_profit@7] + ┌───── Stage 4 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(call_center.cc_call_center_sk AS Float64)@1, cs_call_center_sk@0)], projection=[cs_warehouse_sk@3, cs_order_number@4, cs_ext_ship_cost@5, cs_net_profit@6] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_ship_date_sk@0)], projection=[cs_ship_addr_sk@3, cs_call_center_sk@4, cs_warehouse_sk@5, cs_order_number@6, cs_ext_ship_cost@7, cs_net_profit@8] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer_address.ca_address_sk AS Float64)@1, cs_ship_addr_sk@0)], projection=[cs_call_center_sk@3, cs_warehouse_sk@4, cs_order_number@5, cs_ext_ship_cost@6, cs_net_profit@7] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_ship_date_sk, cs_ship_addr_sk, cs_call_center_sk, cs_warehouse_sk, cs_order_number, cs_ext_ship_cost, cs_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_ship_date_sk@0)], projection=[cs_ship_addr_sk@3, cs_call_center_sk@4, cs_warehouse_sk@5, cs_order_number@6, cs_ext_ship_cost@7, cs_net_profit@8] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_ship_date_sk, cs_ship_addr_sk, cs_call_center_sk, cs_warehouse_sk, cs_order_number, cs_ext_ship_cost, cs_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -2026,13 +2058,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [i_item_id@0 ASC, i_item_desc@1 ASC, s_state@2 ASC], fetch=100 - │ [Stage 11] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 11] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC, i_item_desc@1 ASC, s_state@2 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, s_state@2 as s_state, count(store_sales.ss_quantity)@3 as store_sales_quantitycount, avg(store_sales.ss_quantity)@4 as store_sales_quantityave, stddev(store_sales.ss_quantity)@5 as store_sales_quantitystdev, stddev(store_sales.ss_quantity)@5 / avg(store_sales.ss_quantity)@4 as store_sales_quantitycov, count(store_returns.sr_return_quantity)@6 as store_returns_quantitycount, avg(store_returns.sr_return_quantity)@7 as store_returns_quantityave, stddev(store_returns.sr_return_quantity)@8 as store_returns_quantitystdev, stddev(store_returns.sr_return_quantity)@8 / avg(store_returns.sr_return_quantity)@7 as store_returns_quantitycov, count(catalog_sales.cs_quantity)@9 as catalog_sales_quantitycount, avg(catalog_sales.cs_quantity)@10 as catalog_sales_quantityave, stddev(catalog_sales.cs_quantity)@11 as catalog_sales_quantitystdev, stddev(catalog_sales.cs_quantity)@11 / avg(catalog_sales.cs_quantity)@10 as catalog_sales_quantitycov] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, s_state@2 as s_state], aggr=[count(store_sales.ss_quantity), avg(store_sales.ss_quantity), stddev(store_sales.ss_quantity), count(store_returns.sr_return_quantity), avg(store_returns.sr_return_quantity), stddev(store_returns.sr_return_quantity), count(catalog_sales.cs_quantity), avg(catalog_sales.cs_quantity), stddev(catalog_sales.cs_quantity)] - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 11 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [i_item_id@0 ASC, i_item_desc@1 ASC, s_state@2 ASC] + │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC, i_item_desc@1 ASC, s_state@2 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, s_state@2 as s_state, count(store_sales.ss_quantity)@3 as store_sales_quantitycount, avg(store_sales.ss_quantity)@4 as store_sales_quantityave, stddev(store_sales.ss_quantity)@5 as store_sales_quantitystdev, stddev(store_sales.ss_quantity)@5 / avg(store_sales.ss_quantity)@4 as store_sales_quantitycov, count(store_returns.sr_return_quantity)@6 as store_returns_quantitycount, avg(store_returns.sr_return_quantity)@7 as store_returns_quantityave, stddev(store_returns.sr_return_quantity)@8 as store_returns_quantitystdev, stddev(store_returns.sr_return_quantity)@8 / avg(store_returns.sr_return_quantity)@7 as store_returns_quantitycov, count(catalog_sales.cs_quantity)@9 as catalog_sales_quantitycount, avg(catalog_sales.cs_quantity)@10 as catalog_sales_quantityave, stddev(catalog_sales.cs_quantity)@11 as catalog_sales_quantitystdev, stddev(catalog_sales.cs_quantity)@11 / avg(catalog_sales.cs_quantity)@10 as catalog_sales_quantitycov] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, s_state@2 as s_state], aggr=[count(store_sales.ss_quantity), avg(store_sales.ss_quantity), stddev(store_sales.ss_quantity), count(store_returns.sr_return_quantity), avg(store_returns.sr_return_quantity), stddev(store_returns.sr_return_quantity), count(catalog_sales.cs_quantity), avg(catalog_sales.cs_quantity), stddev(catalog_sales.cs_quantity)] + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, s_state@2], 6), input_partitions=3 @@ -2124,13 +2157,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [ca_country@1 ASC, ca_state@2 ASC, ca_county@3 ASC, i_item_id@0 ASC], fetch=100 - │ [Stage 10] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 10] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[ca_country@1 ASC, ca_state@2 ASC, ca_county@3 ASC, i_item_id@0 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[i_item_id@0 as i_item_id, ca_country@1 as ca_country, ca_state@2 as ca_state, ca_county@3 as ca_county, avg(catalog_sales.cs_quantity)@5 as agg1, avg(catalog_sales.cs_list_price)@6 as agg2, avg(catalog_sales.cs_coupon_amt)@7 as agg3, avg(catalog_sales.cs_sales_price)@8 as agg4, avg(catalog_sales.cs_net_profit)@9 as agg5, avg(customer.c_birth_year)@10 as agg6, avg(cd1.cd_dep_count)@11 as agg7] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, ca_country@1 as ca_country, ca_state@2 as ca_state, ca_county@3 as ca_county, __grouping_id@4 as __grouping_id], aggr=[avg(catalog_sales.cs_quantity), avg(catalog_sales.cs_list_price), avg(catalog_sales.cs_coupon_amt), avg(catalog_sales.cs_sales_price), avg(catalog_sales.cs_net_profit), avg(customer.c_birth_year), avg(cd1.cd_dep_count)] - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 10 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [ca_country@1 ASC, ca_state@2 ASC, ca_county@3 ASC, i_item_id@0 ASC] + │ SortExec: TopK(fetch=100), expr=[ca_country@1 ASC, ca_state@2 ASC, ca_county@3 ASC, i_item_id@0 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[i_item_id@0 as i_item_id, ca_country@1 as ca_country, ca_state@2 as ca_state, ca_county@3 as ca_county, avg(catalog_sales.cs_quantity)@5 as agg1, avg(catalog_sales.cs_list_price)@6 as agg2, avg(catalog_sales.cs_coupon_amt)@7 as agg3, avg(catalog_sales.cs_sales_price)@8 as agg4, avg(catalog_sales.cs_net_profit)@9 as agg5, avg(customer.c_birth_year)@10 as agg6, avg(cd1.cd_dep_count)@11 as agg7] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, ca_country@1 as ca_country, ca_state@2 as ca_state, ca_county@3 as ca_county, __grouping_id@4 as __grouping_id], aggr=[avg(catalog_sales.cs_quantity), avg(catalog_sales.cs_list_price), avg(catalog_sales.cs_coupon_amt), avg(catalog_sales.cs_sales_price), avg(catalog_sales.cs_net_profit), avg(customer.c_birth_year), avg(cd1.cd_dep_count)] + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([i_item_id@0, ca_country@1, ca_state@2, ca_county@3, __grouping_id@4], 6), input_partitions=3 @@ -2215,13 +2249,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [ext_price@4 DESC, brand@1 ASC NULLS LAST, brand_id@0 ASC NULLS LAST, i_manufact_id@2 ASC NULLS LAST, i_manufact@3 ASC NULLS LAST], fetch=100 - │ [Stage 8] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 8] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[ext_price@4 DESC, brand@1 ASC NULLS LAST, brand_id@0 ASC NULLS LAST, i_manufact_id@2 ASC NULLS LAST, i_manufact@3 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[i_brand_id@1 as brand_id, i_brand@0 as brand, i_manufact_id@2 as i_manufact_id, i_manufact@3 as i_manufact, sum(store_sales.ss_ext_sales_price)@4 as ext_price] - │ AggregateExec: mode=FinalPartitioned, gby=[i_brand@0 as i_brand, i_brand_id@1 as i_brand_id, i_manufact_id@2 as i_manufact_id, i_manufact@3 as i_manufact], aggr=[sum(store_sales.ss_ext_sales_price)] - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 8 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [ext_price@4 DESC, brand@1 ASC NULLS LAST, brand_id@0 ASC NULLS LAST, i_manufact_id@2 ASC NULLS LAST, i_manufact@3 ASC NULLS LAST] + │ SortExec: TopK(fetch=100), expr=[ext_price@4 DESC, brand@1 ASC NULLS LAST, brand_id@0 ASC NULLS LAST, i_manufact_id@2 ASC NULLS LAST, i_manufact@3 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[i_brand_id@1 as brand_id, i_brand@0 as brand, i_manufact_id@2 as i_manufact_id, i_manufact@3 as i_manufact, sum(store_sales.ss_ext_sales_price)@4 as ext_price] + │ AggregateExec: mode=FinalPartitioned, gby=[i_brand@0 as i_brand, i_brand_id@1 as i_brand_id, i_manufact_id@2 as i_manufact_id, i_manufact@3 as i_manufact], aggr=[sum(store_sales.ss_ext_sales_price)] + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([i_brand@0, i_brand_id@1, i_manufact_id@2, i_manufact@3], 6), input_partitions=3 @@ -2337,15 +2372,16 @@ mod tests { assert_snapshot!(display, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [w_warehouse_name@0 ASC, i_item_id@1 ASC], fetch=100 - │ [Stage 5] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 5] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[w_warehouse_name@0 ASC, i_item_id@1 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[w_warehouse_name@0 as w_warehouse_name, i_item_id@1 as i_item_id, sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@2 as inv_before, sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@3 as inv_after] - │ FilterExec: __common_expr_4@0 >= 0.6666666666666666 AND __common_expr_4@0 <= 1.5, projection=[w_warehouse_name@1, i_item_id@2, sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@3, sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@4] - │ ProjectionExec: expr=[CASE WHEN sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@2 > 0 THEN sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@3 / sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@2 END as __common_expr_4, w_warehouse_name@0 as w_warehouse_name, i_item_id@1 as i_item_id, sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@2 as sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END), sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@3 as sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)] - │ AggregateExec: mode=FinalPartitioned, gby=[w_warehouse_name@0 as w_warehouse_name, i_item_id@1 as i_item_id], aggr=[sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END), sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 5 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [w_warehouse_name@0 ASC, i_item_id@1 ASC] + │ SortExec: TopK(fetch=100), expr=[w_warehouse_name@0 ASC, i_item_id@1 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[w_warehouse_name@0 as w_warehouse_name, i_item_id@1 as i_item_id, sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@2 as inv_before, sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@3 as inv_after] + │ FilterExec: __common_expr_4@0 >= 0.6666666666666666 AND __common_expr_4@0 <= 1.5, projection=[w_warehouse_name@1, i_item_id@2, sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@3, sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@4] + │ ProjectionExec: expr=[CASE WHEN sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@2 > 0 THEN sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@3 / sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@2 END as __common_expr_4, w_warehouse_name@0 as w_warehouse_name, i_item_id@1 as i_item_id, sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@2 as sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END), sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)@3 as sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)] + │ AggregateExec: mode=FinalPartitioned, gby=[w_warehouse_name@0 as w_warehouse_name, i_item_id@1 as i_item_id], aggr=[sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END), sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN inventory.inv_quantity_on_hand ELSE Int64(0) END)] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([w_warehouse_name@0, i_item_id@1], 6), input_partitions=2 @@ -2393,13 +2429,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [qoh@4 ASC, i_product_name@0 ASC, i_brand@1 ASC, i_class@2 ASC, i_category@3 ASC], fetch=100 - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[qoh@4 ASC, i_product_name@0 ASC, i_brand@1 ASC, i_class@2 ASC, i_category@3 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[i_product_name@0 as i_product_name, i_brand@1 as i_brand, i_class@2 as i_class, i_category@3 as i_category, avg(inventory.inv_quantity_on_hand)@5 as qoh] - │ AggregateExec: mode=FinalPartitioned, gby=[i_product_name@0 as i_product_name, i_brand@1 as i_brand, i_class@2 as i_class, i_category@3 as i_category, __grouping_id@4 as __grouping_id], aggr=[avg(inventory.inv_quantity_on_hand)] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 4 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [qoh@4 ASC, i_product_name@0 ASC, i_brand@1 ASC, i_class@2 ASC, i_category@3 ASC] + │ SortExec: TopK(fetch=100), expr=[qoh@4 ASC, i_product_name@0 ASC, i_brand@1 ASC, i_class@2 ASC, i_category@3 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[i_product_name@0 as i_product_name, i_brand@1 as i_brand, i_class@2 as i_class, i_category@3 as i_category, avg(inventory.inv_quantity_on_hand)@5 as qoh] + │ AggregateExec: mode=FinalPartitioned, gby=[i_product_name@0 as i_product_name, i_brand@1 as i_brand, i_class@2 as i_class, i_category@3 as i_category, __grouping_id@4 as __grouping_id], aggr=[avg(inventory.inv_quantity_on_hand)] + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([i_product_name@0, i_brand@1, i_class@2, i_category@3, __grouping_id@4], 6), input_partitions=2 @@ -2435,17 +2472,18 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c_last_name@0 ASC, c_first_name@1 ASC, sales@2 ASC], fetch=100 - │ [Stage 33] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 33] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 33 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[c_last_name@0 ASC, c_first_name@1 ASC, sales@2 ASC], preserve_partitioning=[true] - │ InterleaveExec - │ ProjectionExec: expr=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, sum(catalog_sales.cs_quantity * catalog_sales.cs_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(catalog_sales.cs_quantity * catalog_sales.cs_list_price)] - │ [Stage 16] => 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)] - │ [Stage 32] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 33 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [c_last_name@0 ASC, c_first_name@1 ASC, sales@2 ASC] + │ SortExec: TopK(fetch=100), expr=[c_last_name@0 ASC, c_first_name@1 ASC, sales@2 ASC], preserve_partitioning=[true] + │ InterleaveExec + │ ProjectionExec: expr=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, sum(catalog_sales.cs_quantity * catalog_sales.cs_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(catalog_sales.cs_quantity * catalog_sales.cs_list_price)] + │ [Stage 16] => 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)] + │ [Stage 32] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 16 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1], 6), input_partitions=3 @@ -2532,17 +2570,18 @@ mod tests { │ ProjectionExec: expr=[max(sq2.csales)@0 as tpcds_cmax] │ AggregateExec: mode=Final, gby=[], aggr=[max(sq2.csales)] │ CoalescePartitionsExec - │ [Stage 12] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 12] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 │ ProjectionExec: expr=[ss_quantity@1 as ss_quantity, ss_sales_price@2 as ss_sales_price, c_customer_sk@0 as c_customer_sk] │ 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 13] => NetworkShuffleExec: output_partitions=3, input_tasks=2 │ [Stage 14] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ AggregateExec: mode=Partial, gby=[], aggr=[max(sq2.csales)] - │ ProjectionExec: expr=[sum(store_sales.ss_quantity * store_sales.ss_sales_price)@1 as csales] - │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_sk@0 as c_customer_sk], aggr=[sum(store_sales.ss_quantity * store_sales.ss_sales_price)] - │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 12 ── Tasks: t0:[p0] t1:[p1] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[max(sq2.csales)] + │ ProjectionExec: expr=[sum(store_sales.ss_quantity * store_sales.ss_sales_price)@1 as csales] + │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_sk@0 as c_customer_sk], aggr=[sum(store_sales.ss_quantity * store_sales.ss_sales_price)] + │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([c_customer_sk@0], 6), input_partitions=3 @@ -2672,17 +2711,18 @@ mod tests { │ ProjectionExec: expr=[max(sq2.csales)@0 as tpcds_cmax] │ AggregateExec: mode=Final, gby=[], aggr=[max(sq2.csales)] │ CoalescePartitionsExec - │ [Stage 28] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 28] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 │ ProjectionExec: expr=[ss_quantity@1 as ss_quantity, ss_sales_price@2 as ss_sales_price, c_customer_sk@0 as c_customer_sk] │ 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 29] => NetworkShuffleExec: output_partitions=3, input_tasks=2 │ [Stage 30] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 28 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ AggregateExec: mode=Partial, gby=[], aggr=[max(sq2.csales)] - │ ProjectionExec: expr=[sum(store_sales.ss_quantity * store_sales.ss_sales_price)@1 as csales] - │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_sk@0 as c_customer_sk], aggr=[sum(store_sales.ss_quantity * store_sales.ss_sales_price)] - │ [Stage 27] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 28 ── Tasks: t0:[p0] t1:[p1] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[max(sq2.csales)] + │ ProjectionExec: expr=[sum(store_sales.ss_quantity * store_sales.ss_sales_price)@1 as csales] + │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_sk@0 as c_customer_sk], aggr=[sum(store_sales.ss_quantity * store_sales.ss_sales_price)] + │ [Stage 27] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 27 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([c_customer_sk@0], 6), input_partitions=3 @@ -2745,7 +2785,7 @@ mod tests { │ ProjectionExec: expr=[CAST(0.05 * CAST(avg(ssales.netpaid)@0 AS Float64) AS Decimal128(38, 15)) as Float64(0.05) * avg(ssales.netpaid)] │ AggregateExec: mode=Final, gby=[], aggr=[avg(ssales.netpaid)] │ CoalescePartitionsExec - │ [Stage 16] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 16] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1, s_store_name@2], 3), input_partitions=3 @@ -2812,11 +2852,12 @@ mod tests { │ 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 16 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ AggregateExec: mode=Partial, gby=[], aggr=[avg(ssales.netpaid)] - │ ProjectionExec: expr=[sum(store_sales.ss_net_paid)@10 as netpaid] - │ AggregateExec: mode=FinalPartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, s_store_name@2 as s_store_name, ca_state@3 as ca_state, s_state@4 as s_state, i_color@5 as i_color, i_current_price@6 as i_current_price, i_manager_id@7 as i_manager_id, i_units@8 as i_units, i_size@9 as i_size], aggr=[sum(store_sales.ss_net_paid)] - │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 16 ── Tasks: t0:[p0] t1:[p1] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(ssales.netpaid)] + │ ProjectionExec: expr=[sum(store_sales.ss_net_paid)@10 as netpaid] + │ AggregateExec: mode=FinalPartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, s_store_name@2 as s_store_name, ca_state@3 as ca_state, s_state@4 as s_state, i_color@5 as i_color, i_current_price@6 as i_current_price, i_manager_id@7 as i_manager_id, i_units@8 as i_units, i_size@9 as i_size], aggr=[sum(store_sales.ss_net_paid)] + │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 15 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1, s_store_name@2, ca_state@3, s_state@4, i_color@5, i_current_price@6, i_manager_id@7, i_units@8, i_size@9], 6), input_partitions=3 @@ -2884,13 +2925,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [i_item_id@0 ASC NULLS LAST, i_item_desc@1 ASC NULLS LAST, s_store_id@2 ASC NULLS LAST, s_store_name@3 ASC NULLS LAST], fetch=100 - │ [Stage 11] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 11] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC NULLS LAST, i_item_desc@1 ASC NULLS LAST, s_store_id@2 ASC NULLS LAST, s_store_name@3 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, s_store_id@2 as s_store_id, s_store_name@3 as s_store_name, sum(store_sales.ss_net_profit)@4 as store_sales_profit, sum(store_returns.sr_net_loss)@5 as store_returns_loss, sum(catalog_sales.cs_net_profit)@6 as catalog_sales_profit] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, s_store_id@2 as s_store_id, s_store_name@3 as s_store_name], aggr=[sum(store_sales.ss_net_profit), sum(store_returns.sr_net_loss), sum(catalog_sales.cs_net_profit)] - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 11 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [i_item_id@0 ASC NULLS LAST, i_item_desc@1 ASC NULLS LAST, s_store_id@2 ASC NULLS LAST, s_store_name@3 ASC NULLS LAST] + │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC NULLS LAST, i_item_desc@1 ASC NULLS LAST, s_store_id@2 ASC NULLS LAST, s_store_name@3 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, s_store_id@2 as s_store_id, s_store_name@3 as s_store_name, sum(store_sales.ss_net_profit)@4 as store_sales_profit, sum(store_returns.sr_net_loss)@5 as store_returns_loss, sum(catalog_sales.cs_net_profit)@6 as catalog_sales_profit] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, s_store_id@2 as s_store_id, s_store_name@3 as s_store_name], aggr=[sum(store_sales.ss_net_profit), sum(store_returns.sr_net_loss), sum(catalog_sales.cs_net_profit)] + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, s_store_id@2, s_store_name@3], 6), input_partitions=3 @@ -2982,13 +3024,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [i_item_id@0 ASC NULLS LAST], fetch=100 - │ [Stage 7] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 7] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[i_item_id@0 as i_item_id, avg(catalog_sales.cs_quantity)@1 as agg1, avg(catalog_sales.cs_list_price)@2 as agg2, avg(catalog_sales.cs_coupon_amt)@3 as agg3, avg(catalog_sales.cs_sales_price)@4 as agg4] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[avg(catalog_sales.cs_quantity), avg(catalog_sales.cs_list_price), avg(catalog_sales.cs_coupon_amt), avg(catalog_sales.cs_sales_price)] - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 7 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [i_item_id@0 ASC NULLS LAST] + │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[i_item_id@0 as i_item_id, avg(catalog_sales.cs_quantity)@1 as agg1, avg(catalog_sales.cs_list_price)@2 as agg2, avg(catalog_sales.cs_coupon_amt)@3 as agg3, avg(catalog_sales.cs_sales_price)@4 as agg4] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[avg(catalog_sales.cs_quantity), avg(catalog_sales.cs_list_price), avg(catalog_sales.cs_coupon_amt), avg(catalog_sales.cs_sales_price)] + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([i_item_id@0], 6), input_partitions=3 @@ -3049,21 +3092,22 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [i_item_id@0 ASC, s_state@1 ASC], fetch=100 - │ [Stage 19] => NetworkCoalesceExec: output_partitions=12, input_tasks=4 + │ [Stage 19] => NetworkCoalesceExec: output_partitions=4, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 19 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] t3:[p9..p11] - │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC], preserve_partitioning=[true] - │ DistributedUnionExec: t0:[c0] t1:[c1(0/2)] t2:[c1(1/2)] t3:[c2] - │ ProjectionExec: expr=[i_item_id@0 as i_item_id, CAST(s_state@1 AS Utf8) as s_state, 0 as g_state, avg(results.agg1)@2 as agg1, avg(results.agg2)@3 as agg2, avg(results.agg3)@4 as agg3, avg(results.agg4)@5 as agg4] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, s_state@1 as s_state], aggr=[avg(results.agg1), avg(results.agg2), avg(results.agg3), avg(results.agg4)], ordering_mode=PartiallySorted([1]) - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ 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] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[avg(results.agg1), avg(results.agg2), avg(results.agg3), avg(results.agg4)] - │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ 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] - │ AggregateExec: mode=Final, gby=[], aggr=[avg(results.agg1), avg(results.agg2), avg(results.agg3), avg(results.agg4)] - │ CoalescePartitionsExec - │ [Stage 18] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + ┌───── Stage 19 ── Tasks: t0:[p0] t1:[p1] t2:[p2] t3:[p3] + │ SortPreservingMergeExec: [i_item_id@0 ASC, s_state@1 ASC] + │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC], preserve_partitioning=[true] + │ DistributedUnionExec: t0:[c0] t1:[c1(0/2)] t2:[c1(1/2)] t3:[c2] + │ ProjectionExec: expr=[i_item_id@0 as i_item_id, CAST(s_state@1 AS Utf8) as s_state, 0 as g_state, avg(results.agg1)@2 as agg1, avg(results.agg2)@3 as agg2, avg(results.agg3)@4 as agg3, avg(results.agg4)@5 as agg4] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, s_state@1 as s_state], aggr=[avg(results.agg1), avg(results.agg2), avg(results.agg3), avg(results.agg4)], ordering_mode=PartiallySorted([1]) + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ 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] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[avg(results.agg1), avg(results.agg2), avg(results.agg3), avg(results.agg4)] + │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ 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] + │ AggregateExec: mode=Final, gby=[], aggr=[avg(results.agg1), avg(results.agg2), avg(results.agg3), avg(results.agg4)] + │ CoalescePartitionsExec + │ [Stage 18] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ RepartitionExec: partitioning=Hash([i_item_id@0, s_state@1], 3), input_partitions=3 @@ -3168,15 +3212,16 @@ mod tests { │ 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 18 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ AggregateExec: mode=Partial, gby=[], aggr=[avg(results.agg1), avg(results.agg2), avg(results.agg3), avg(results.agg4)] - │ ProjectionExec: expr=[ss_quantity@0 as agg1, ss_list_price@1 as agg2, ss_coupon_amt@3 as agg3, ss_sales_price@2 as agg4] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_quantity@1, ss_list_price@2, ss_sales_price@3, ss_coupon_amt@4] - │ CoalescePartitionsExec - │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ 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, predicate=DynamicFilter [ empty ] + ┌───── Stage 18 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(results.agg1), avg(results.agg2), avg(results.agg3), avg(results.agg4)] + │ ProjectionExec: expr=[ss_quantity@0 as agg1, ss_list_price@1 as agg2, ss_coupon_amt@3 as agg3, ss_sales_price@2 as agg4] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_quantity@1, ss_list_price@2, ss_sales_price@3, ss_coupon_amt@4] + │ CoalescePartitionsExec + │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ 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, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 17 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -3232,70 +3277,76 @@ 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=3 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=3, 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=3 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=3, 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=3 + │ [Stage 3] => NetworkCoalesceExec: output_partitions=3, 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=3 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=3, 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=3 + │ [Stage 5] => NetworkCoalesceExec: output_partitions=3, 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=3 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── 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)] - │ 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,__,__,__,__] 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@10 >= 26 AND ss_quantity@10 <= 30 AND (ss_list_price@12 >= Some(15400),5,2 AND ss_list_price@12 <= Some(16400),5,2 OR ss_coupon_amt@19 >= Some(732600),7,2 AND ss_coupon_amt@19 <= Some(832600),7,2 OR ss_wholesale_cost@11 >= Some(700),5,2 AND ss_wholesale_cost@11 <= 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 1 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] + │ 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,__,__,__,__] 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@10 >= 26 AND ss_quantity@10 <= 30 AND (ss_list_price@12 >= Some(15400),5,2 AND ss_list_price@12 <= Some(16400),5,2 OR ss_coupon_amt@19 >= Some(732600),7,2 AND ss_coupon_amt@19 <= Some(832600),7,2 OR ss_wholesale_cost@11 >= Some(700),5,2 AND ss_wholesale_cost@11 <= 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..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)] - │ 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,__,__,__,__] 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@10 >= 21 AND ss_quantity@10 <= 25 AND (ss_list_price@12 >= Some(12200),5,2 AND ss_list_price@12 <= Some(13200),5,2 OR ss_coupon_amt@19 >= Some(83600),7,2 AND ss_coupon_amt@19 <= Some(183600),7,2 OR ss_wholesale_cost@11 >= Some(1700),5,2 AND ss_wholesale_cost@11 <= 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 2 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] + │ 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,__,__,__,__] 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@10 >= 21 AND ss_quantity@10 <= 25 AND (ss_list_price@12 >= Some(12200),5,2 AND ss_list_price@12 <= Some(13200),5,2 OR ss_coupon_amt@19 >= Some(83600),7,2 AND ss_coupon_amt@19 <= Some(183600),7,2 OR ss_wholesale_cost@11 >= Some(1700),5,2 AND ss_wholesale_cost@11 <= 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..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)] - │ 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,__,__,__,__] 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@10 >= 16 AND ss_quantity@10 <= 20 AND (ss_list_price@12 >= Some(13500),5,2 AND ss_list_price@12 <= Some(14500),5,2 OR ss_coupon_amt@19 >= Some(607100),7,2 AND ss_coupon_amt@19 <= Some(707100),7,2 OR ss_wholesale_cost@11 >= Some(3800),5,2 AND ss_wholesale_cost@11 <= 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 3 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] + │ 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,__,__,__,__] 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@10 >= 16 AND ss_quantity@10 <= 20 AND (ss_list_price@12 >= Some(13500),5,2 AND ss_list_price@12 <= Some(14500),5,2 OR ss_coupon_amt@19 >= Some(607100),7,2 AND ss_coupon_amt@19 <= Some(707100),7,2 OR ss_wholesale_cost@11 >= Some(3800),5,2 AND ss_wholesale_cost@11 <= 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..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)] - │ 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,__,__,__,__] 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@10 >= 11 AND ss_quantity@10 <= 15 AND (ss_list_price@12 >= Some(14200),5,2 AND ss_list_price@12 <= Some(15200),5,2 OR ss_coupon_amt@19 >= Some(1221400),7,2 AND ss_coupon_amt@19 <= Some(1321400),7,2 OR ss_wholesale_cost@11 >= Some(7900),5,2 AND ss_wholesale_cost@11 <= 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 4 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] + │ 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,__,__,__,__] 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@10 >= 11 AND ss_quantity@10 <= 15 AND (ss_list_price@12 >= Some(14200),5,2 AND ss_list_price@12 <= Some(15200),5,2 OR ss_coupon_amt@19 >= Some(1221400),7,2 AND ss_coupon_amt@19 <= Some(1321400),7,2 OR ss_wholesale_cost@11 >= Some(7900),5,2 AND ss_wholesale_cost@11 <= 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..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)] - │ 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,__,__,__,__] 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@10 >= 0 AND ss_quantity@10 <= 5 AND (ss_list_price@12 >= Some(800),5,2 AND ss_list_price@12 <= Some(1800),5,2 OR ss_coupon_amt@19 >= Some(45900),7,2 AND ss_coupon_amt@19 <= Some(145900),7,2 OR ss_wholesale_cost@11 >= Some(5700),5,2 AND ss_wholesale_cost@11 <= 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 5 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] + │ 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,__,__,__,__] 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@10 >= 0 AND ss_quantity@10 <= 5 AND (ss_list_price@12 >= Some(800),5,2 AND ss_list_price@12 <= Some(1800),5,2 OR ss_coupon_amt@19 >= Some(45900),7,2 AND ss_coupon_amt@19 <= Some(145900),7,2 OR ss_wholesale_cost@11 >= Some(5700),5,2 AND ss_wholesale_cost@11 <= 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..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)] - │ 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,__,__,__,__] 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@10 >= 6 AND ss_quantity@10 <= 10 AND (ss_list_price@12 >= Some(9000),5,2 AND ss_list_price@12 <= Some(10000),5,2 OR ss_coupon_amt@19 >= Some(232300),7,2 AND ss_coupon_amt@19 <= Some(332300),7,2 OR ss_wholesale_cost@11 >= Some(3100),5,2 AND ss_wholesale_cost@11 <= 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=[] + ┌───── Stage 6 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(store_sales.ss_list_price), count(store_sales.ss_list_price), count(DISTINCT store_sales.ss_list_price)] + │ 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,__,__,__,__] 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@10 >= 6 AND ss_quantity@10 <= 10 AND (ss_list_price@12 >= Some(9000),5,2 AND ss_list_price@12 <= Some(10000),5,2 OR ss_coupon_amt@19 >= Some(232300),7,2 AND ss_coupon_amt@19 <= Some(332300),7,2 OR ss_wholesale_cost@11 >= Some(3100),5,2 AND ss_wholesale_cost@11 <= 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=[] └────────────────────────────────────────────────── "); Ok(()) @@ -3306,13 +3357,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [i_item_id@0 ASC NULLS LAST, i_item_desc@1 ASC NULLS LAST, s_store_id@2 ASC NULLS LAST, s_store_name@3 ASC NULLS LAST], fetch=100 - │ [Stage 11] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 11] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC NULLS LAST, i_item_desc@1 ASC NULLS LAST, s_store_id@2 ASC NULLS LAST, s_store_name@3 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, s_store_id@2 as s_store_id, s_store_name@3 as s_store_name, sum(store_sales.ss_quantity)@4 as store_sales_quantity, sum(store_returns.sr_return_quantity)@5 as store_returns_quantity, sum(catalog_sales.cs_quantity)@6 as catalog_sales_quantity] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, s_store_id@2 as s_store_id, s_store_name@3 as s_store_name], aggr=[sum(store_sales.ss_quantity), sum(store_returns.sr_return_quantity), sum(catalog_sales.cs_quantity)] - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 11 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [i_item_id@0 ASC NULLS LAST, i_item_desc@1 ASC NULLS LAST, s_store_id@2 ASC NULLS LAST, s_store_name@3 ASC NULLS LAST] + │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC NULLS LAST, i_item_desc@1 ASC NULLS LAST, s_store_id@2 ASC NULLS LAST, s_store_name@3 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, s_store_id@2 as s_store_id, s_store_name@3 as s_store_name, sum(store_sales.ss_quantity)@4 as store_sales_quantity, sum(store_returns.sr_return_quantity)@5 as store_returns_quantity, sum(catalog_sales.cs_quantity)@6 as catalog_sales_quantity] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, s_store_id@2 as s_store_id, s_store_name@3 as s_store_name], aggr=[sum(store_sales.ss_quantity), sum(store_returns.sr_return_quantity), sum(catalog_sales.cs_quantity)] + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, s_store_id@2, s_store_name@3], 6), input_partitions=3 @@ -3411,18 +3463,19 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [ca_county@0 ASC NULLS LAST] - │ [Stage 24] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 24] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 24 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: expr=[ca_county@0 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[ca_county@2 as ca_county, d_year@3 as d_year, __common_expr_1@0 / CAST(web_sales@6 AS Float64) as web_q1_q2_increase, __common_expr_2@1 / CAST(store_sales@4 AS Float64) as store_q1_q2_increase, CAST(web_sales@7 AS Float64) / __common_expr_1@0 as web_q2_q3_increase, CAST(store_sales@5 AS Float64) / __common_expr_2@1 as store_q2_q3_increase] - │ ProjectionExec: expr=[CAST(web_sales@6 AS Float64) as __common_expr_1, CAST(store_sales@3 AS Float64) as __common_expr_2, ca_county@0 as ca_county, d_year@1 as d_year, store_sales@2 as store_sales, store_sales@4 as store_sales, web_sales@5 as web_sales, web_sales@7 as web_sales] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_county@5, ca_county@0)], filter=CASE WHEN web_sales@2 > Some(0),17,2 THEN CAST(web_sales@3 AS Float64) / CAST(web_sales@2 AS Float64) END > CASE WHEN store_sales@0 > Some(0),17,2 THEN CAST(store_sales@1 AS Float64) / CAST(store_sales@0 AS Float64) END, projection=[ca_county@0, d_year@1, store_sales@2, store_sales@3, store_sales@4, web_sales@6, web_sales@7, web_sales@9] - │ CoalescePartitionsExec - │ [Stage 20] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[ca_county@0 as ca_county, sum(web_sales.ws_ext_sales_price)@3 as web_sales] - │ AggregateExec: mode=FinalPartitioned, gby=[ca_county@0 as ca_county, d_qoy@1 as d_qoy, d_year@2 as d_year], aggr=[sum(web_sales.ws_ext_sales_price)], ordering_mode=PartiallySorted([1, 2]) - │ [Stage 23] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 24 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [ca_county@0 ASC NULLS LAST] + │ SortExec: expr=[ca_county@0 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[ca_county@2 as ca_county, d_year@3 as d_year, __common_expr_1@0 / CAST(web_sales@6 AS Float64) as web_q1_q2_increase, __common_expr_2@1 / CAST(store_sales@4 AS Float64) as store_q1_q2_increase, CAST(web_sales@7 AS Float64) / __common_expr_1@0 as web_q2_q3_increase, CAST(store_sales@5 AS Float64) / __common_expr_2@1 as store_q2_q3_increase] + │ ProjectionExec: expr=[CAST(web_sales@6 AS Float64) as __common_expr_1, CAST(store_sales@3 AS Float64) as __common_expr_2, ca_county@0 as ca_county, d_year@1 as d_year, store_sales@2 as store_sales, store_sales@4 as store_sales, web_sales@5 as web_sales, web_sales@7 as web_sales] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_county@5, ca_county@0)], filter=CASE WHEN web_sales@2 > Some(0),17,2 THEN CAST(web_sales@3 AS Float64) / CAST(web_sales@2 AS Float64) END > CASE WHEN store_sales@0 > Some(0),17,2 THEN CAST(store_sales@1 AS Float64) / CAST(store_sales@0 AS Float64) END, projection=[ca_county@0, d_year@1, store_sales@2, store_sales@3, store_sales@4, web_sales@6, web_sales@7, web_sales@9] + │ CoalescePartitionsExec + │ [Stage 20] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ ProjectionExec: expr=[ca_county@0 as ca_county, sum(web_sales.ws_ext_sales_price)@3 as web_sales] + │ AggregateExec: mode=FinalPartitioned, gby=[ca_county@0 as ca_county, d_qoy@1 as d_qoy, d_year@2 as d_year], aggr=[sum(web_sales.ws_ext_sales_price)], ordering_mode=PartiallySorted([1, 2]) + │ [Stage 23] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 20 ── Tasks: t0:[p0..p5] t1:[p6..p11] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 @@ -3647,22 +3700,23 @@ mod tests { │ GlobalLimitExec: skip=0, fetch=100 │ AggregateExec: mode=Final, gby=[], aggr=[sum(catalog_sales.cs_ext_discount_amt)] │ CoalescePartitionsExec - │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[sum(catalog_sales.cs_ext_discount_amt)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@1, i_item_sk@1)], filter=CAST(cs_ext_discount_amt@0 AS Decimal128(30, 15)) > Float64(1.3) * avg(catalog_sales.cs_ext_discount_amt)@1, projection=[cs_ext_discount_amt@2] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_ext_discount_amt@3, i_item_sk@4] + ┌───── Stage 6 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(catalog_sales.cs_ext_discount_amt)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@1, i_item_sk@1)], filter=CAST(cs_ext_discount_amt@0 AS Decimal128(30, 15)) > Float64(1.3) * avg(catalog_sales.cs_ext_discount_amt)@1, projection=[cs_ext_discount_amt@2] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_ext_discount_amt@2 as cs_ext_discount_amt, i_item_sk@0 as i_item_sk] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_item_sk@0, cs_sold_date_sk@1, cs_ext_discount_amt@3] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_discount_amt], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_ext_discount_amt@3, i_item_sk@4] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_ext_discount_amt@2 as cs_ext_discount_amt, i_item_sk@0 as i_item_sk] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_item_sk@0, cs_sold_date_sk@1, cs_ext_discount_amt@3] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_discount_amt], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -3721,7 +3775,7 @@ mod tests { │ AggregateExec: mode=Partial, gby=[i_manufact_id@1 as i_manufact_id], aggr=[sum(store_sales.ss_ext_sales_price)] │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(i_manufact_id@1, i_manufact_id@0)] │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ FilterExec: i_category@0 = Electronics, projection=[i_manufact_id@1] │ 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_category, i_manufact_id], file_type=parquet, predicate=i_category@12 = Electronics, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Electronics AND Electronics <= i_category_max@1, required_guarantees=[i_category in (Electronics)] @@ -3731,7 +3785,7 @@ mod tests { │ AggregateExec: mode=Partial, gby=[i_manufact_id@1 as i_manufact_id], aggr=[sum(catalog_sales.cs_ext_sales_price)] │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(i_manufact_id@1, i_manufact_id@0)] │ CoalescePartitionsExec - │ [Stage 8] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ [Stage 8] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ FilterExec: i_category@0 = Electronics, projection=[i_manufact_id@1] │ 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_category, i_manufact_id], file_type=parquet, predicate=i_category@12 = Electronics, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Electronics AND Electronics <= i_category_max@1, required_guarantees=[i_category in (Electronics)] @@ -3741,18 +3795,19 @@ mod tests { │ AggregateExec: mode=Partial, gby=[i_manufact_id@1 as i_manufact_id], aggr=[sum(web_sales.ws_ext_sales_price)] │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(i_manufact_id@1, i_manufact_id@0)] │ CoalescePartitionsExec - │ [Stage 12] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ [Stage 12] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ FilterExec: i_category@0 = Electronics, projection=[i_manufact_id@1] │ 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_category, i_manufact_id], file_type=parquet, predicate=i_category@12 = Electronics, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Electronics AND Electronics <= i_category_max@1, required_guarantees=[i_category in (Electronics)] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_ext_sales_price@1, i_manufact_id@3] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ 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_manufact_id], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_ext_sales_price@1, i_manufact_id@3] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ 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_manufact_id], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -3781,13 +3836,14 @@ 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@6 = 1998 AND d_moy@8 = 5, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 5 AND 5 <= d_moy_max@5, required_guarantees=[d_moy in (5), d_year in (1998)] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, i_item_sk@0)], projection=[cs_ext_sales_price@1, i_manufact_id@3] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ 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_manufact_id], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 8 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, i_item_sk@0)], projection=[cs_ext_sales_price@1, i_manufact_id@3] + │ CoalescePartitionsExec + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ 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_manufact_id], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -3816,13 +3872,14 @@ 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@6 = 1998 AND d_moy@8 = 5, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 5 AND 5 <= d_moy_max@5, required_guarantees=[d_moy in (5), d_year in (1998)] └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_item_sk@0, i_item_sk@0)], projection=[ws_ext_sales_price@1, i_manufact_id@3] - │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ 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_manufact_id], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 12 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_item_sk@0, i_item_sk@0)], projection=[ws_ext_sales_price@1, i_manufact_id@3] + │ CoalescePartitionsExec + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ 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_manufact_id], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -3860,18 +3917,19 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c_last_name@0 ASC, c_first_name@1 ASC, c_salutation@2 ASC, c_preferred_cust_flag@3 DESC, ss_ticket_number@4 ASC] - │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] - │ SortExec: expr=[c_last_name@0 ASC, c_first_name@1 ASC, c_salutation@2 ASC, c_preferred_cust_flag@3 DESC, ss_ticket_number@4 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[c_last_name@4 as c_last_name, c_first_name@3 as c_first_name, c_salutation@2 as c_salutation, c_preferred_cust_flag@5 as c_preferred_cust_flag, ss_ticket_number@0 as ss_ticket_number, cnt@1 as cnt] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@5)], projection=[ss_ticket_number@0, cnt@2, c_salutation@4, c_first_name@5, c_last_name@6, c_preferred_cust_flag@7] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_salutation@1 as c_salutation, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_preferred_cust_flag@4 as c_preferred_cust_flag, 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_salutation, c_first_name, c_last_name, c_preferred_cust_flag], file_type=parquet + ┌───── Stage 6 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [c_last_name@0 ASC, c_first_name@1 ASC, c_salutation@2 ASC, c_preferred_cust_flag@3 DESC, ss_ticket_number@4 ASC] + │ SortExec: expr=[c_last_name@0 ASC, c_first_name@1 ASC, c_salutation@2 ASC, c_preferred_cust_flag@3 DESC, ss_ticket_number@4 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[c_last_name@4 as c_last_name, c_first_name@3 as c_first_name, c_salutation@2 as c_salutation, c_preferred_cust_flag@5 as c_preferred_cust_flag, ss_ticket_number@0 as ss_ticket_number, cnt@1 as cnt] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@5)], projection=[ss_ticket_number@0, cnt@2, c_salutation@4, c_first_name@5, c_last_name@6, c_preferred_cust_flag@7] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_salutation@1 as c_salutation, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_preferred_cust_flag@4 as c_preferred_cust_flag, 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_salutation, c_first_name, c_last_name, c_preferred_cust_flag], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 @@ -4046,7 +4104,7 @@ mod tests { │ ProjectionExec: expr=[CAST(sum(results.ss_net_profit)@0 AS Float64) / CAST(sum(results.ss_ext_sales_price)@1 AS Float64) as gross_margin, NULL as i_category, NULL as i_class, 1 as t_category, 1 as t_class, 2 as lochierarchy] │ AggregateExec: mode=Final, gby=[], aggr=[sum(results.ss_net_profit), sum(results.ss_ext_sales_price)] │ CoalescePartitionsExec - │ [Stage 15] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 15] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ RepartitionExec: partitioning=Hash([gross_margin@0, i_category@1, i_class@2, t_category@3, t_class@4, lochierarchy@5], 3), input_partitions=3 @@ -4140,11 +4198,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], file_type=parquet, predicate=d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1, required_guarantees=[d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 15 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ AggregateExec: mode=Partial, gby=[], aggr=[sum(results.ss_net_profit), sum(results.ss_ext_sales_price)] - │ ProjectionExec: expr=[sum(store_sales.ss_net_profit)@2 as ss_net_profit, sum(store_sales.ss_ext_sales_price)@3 as ss_ext_sales_price] - │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category, i_class@1 as i_class], aggr=[sum(store_sales.ss_net_profit), sum(store_sales.ss_ext_sales_price)] - │ [Stage 14] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 15 ── Tasks: t0:[p0] t1:[p1] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(results.ss_net_profit), sum(results.ss_ext_sales_price)] + │ ProjectionExec: expr=[sum(store_sales.ss_net_profit)@2 as ss_net_profit, sum(store_sales.ss_ext_sales_price)@3 as ss_ext_sales_price] + │ AggregateExec: mode=FinalPartitioned, gby=[i_category@0 as i_category, i_class@1 as i_class], aggr=[sum(store_sales.ss_net_profit), sum(store_sales.ss_ext_sales_price)] + │ [Stage 14] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 14 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([i_category@0, i_class@1], 6), input_partitions=3 @@ -4192,12 +4251,13 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [i_item_id@0 ASC NULLS LAST], fetch=100 - │ [Stage 5] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 5] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC NULLS LAST], preserve_partitioning=[true] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_current_price@2 as i_current_price], aggr=[] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 5 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [i_item_id@0 ASC NULLS LAST] + │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC NULLS LAST], preserve_partitioning=[true] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_current_price@2 as i_current_price], aggr=[] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, i_current_price@2], 6), input_partitions=2 @@ -4246,21 +4306,22 @@ mod tests { │ GlobalLimitExec: skip=0, fetch=100 │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 12] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 12] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] - │ ProjectionExec: expr=[] - │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(c_last_name@0, c_last_name@0), (c_first_name@1, c_first_name@1), (d_date@2, d_date@2)], NullsEqual: true - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ AggregateExec: mode=SinglePartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] - │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(c_last_name@0, c_last_name@0), (c_first_name@1, c_first_name@1), (d_date@2, d_date@2)], NullsEqual: true - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ AggregateExec: mode=SinglePartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] - │ AggregateExec: mode=FinalPartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] - │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 12 ── Tasks: t0:[p0] t1:[p1] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] + │ ProjectionExec: expr=[] + │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(c_last_name@0, c_last_name@0), (c_first_name@1, c_first_name@1), (d_date@2, d_date@2)], NullsEqual: true + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ AggregateExec: mode=SinglePartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] + │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(c_last_name@0, c_last_name@0), (c_first_name@1, c_first_name@1), (d_date@2, d_date@2)], NullsEqual: true + │ CoalescePartitionsExec + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ AggregateExec: mode=SinglePartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] + │ AggregateExec: mode=FinalPartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] + │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 @@ -4470,13 +4531,14 @@ mod tests { assert_snapshot!(display, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [w_state@0 ASC NULLS LAST, i_item_id@1 ASC NULLS LAST], fetch=100 - │ [Stage 7] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 7] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[w_state@0 ASC NULLS LAST, i_item_id@1 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[w_state@0 as w_state, i_item_id@1 as i_item_id, sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN catalog_sales.cs_sales_price - coalesce(catalog_returns.cr_refunded_cash,Int64(0)) ELSE Int64(0) END)@2 as sales_before, sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN catalog_sales.cs_sales_price - coalesce(catalog_returns.cr_refunded_cash,Int64(0)) ELSE Int64(0) END)@3 as sales_after] - │ AggregateExec: mode=FinalPartitioned, gby=[w_state@0 as w_state, i_item_id@1 as i_item_id], aggr=[sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN catalog_sales.cs_sales_price - coalesce(catalog_returns.cr_refunded_cash,Int64(0)) ELSE Int64(0) END), sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN catalog_sales.cs_sales_price - coalesce(catalog_returns.cr_refunded_cash,Int64(0)) ELSE Int64(0) END)] - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 7 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [w_state@0 ASC NULLS LAST, i_item_id@1 ASC NULLS LAST] + │ SortExec: TopK(fetch=100), expr=[w_state@0 ASC NULLS LAST, i_item_id@1 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[w_state@0 as w_state, i_item_id@1 as i_item_id, sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN catalog_sales.cs_sales_price - coalesce(catalog_returns.cr_refunded_cash,Int64(0)) ELSE Int64(0) END)@2 as sales_before, sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN catalog_sales.cs_sales_price - coalesce(catalog_returns.cr_refunded_cash,Int64(0)) ELSE Int64(0) END)@3 as sales_after] + │ AggregateExec: mode=FinalPartitioned, gby=[w_state@0 as w_state, i_item_id@1 as i_item_id], aggr=[sum(CASE WHEN date_dim.d_date < Utf8("2000-03-11") THEN catalog_sales.cs_sales_price - coalesce(catalog_returns.cr_refunded_cash,Int64(0)) ELSE Int64(0) END), sum(CASE WHEN date_dim.d_date >= Utf8("2000-03-11") THEN catalog_sales.cs_sales_price - coalesce(catalog_returns.cr_refunded_cash,Int64(0)) ELSE Int64(0) END)] + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([w_state@0, i_item_id@1], 6), input_partitions=3 @@ -4577,12 +4639,13 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [sum(store_sales.ss_ext_sales_price)@3 DESC, d_year@0 ASC NULLS LAST, i_category_id@1 ASC NULLS LAST, i_category@2 ASC NULLS LAST], fetch=100 - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[sum(store_sales.ss_ext_sales_price)@3 DESC, i_category_id@1 ASC NULLS LAST, i_category@2 ASC NULLS LAST], preserve_partitioning=[true] - │ AggregateExec: mode=FinalPartitioned, gby=[d_year@0 as d_year, i_category_id@1 as i_category_id, i_category@2 as i_category], aggr=[sum(store_sales.ss_ext_sales_price)], ordering_mode=PartiallySorted([0]) - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 4 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [sum(store_sales.ss_ext_sales_price)@3 DESC, d_year@0 ASC NULLS LAST, i_category_id@1 ASC NULLS LAST, i_category@2 ASC NULLS LAST] + │ SortExec: TopK(fetch=100), expr=[sum(store_sales.ss_ext_sales_price)@3 DESC, i_category_id@1 ASC NULLS LAST, i_category@2 ASC NULLS LAST], preserve_partitioning=[true] + │ AggregateExec: mode=FinalPartitioned, gby=[d_year@0 as d_year, i_category_id@1 as i_category_id, i_category@2 as i_category], aggr=[sum(store_sales.ss_ext_sales_price)], ordering_mode=PartiallySorted([0]) + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([d_year@0, i_category_id@1, i_category@2], 6), input_partitions=3 @@ -4620,13 +4683,14 @@ mod tests { assert_snapshot!(display, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [s_store_name@0 ASC NULLS LAST, s_store_id@1 ASC NULLS LAST, sun_sales@2 ASC NULLS LAST, mon_sales@3 ASC NULLS LAST, tue_sales@4 ASC NULLS LAST, wed_sales@5 ASC NULLS LAST, thu_sales@6 ASC NULLS LAST, fri_sales@7 ASC NULLS LAST, sat_sales@8 ASC NULLS LAST], fetch=100 - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[s_store_name@0 ASC NULLS LAST, s_store_id@1 ASC NULLS LAST, sun_sales@2 ASC NULLS LAST, mon_sales@3 ASC NULLS LAST, tue_sales@4 ASC NULLS LAST, wed_sales@5 ASC NULLS LAST, thu_sales@6 ASC NULLS LAST, fri_sales@7 ASC NULLS LAST, sat_sales@8 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[s_store_name@0 as s_store_name, s_store_id@1 as s_store_id, 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=[s_store_name@0 as s_store_name, s_store_id@1 as s_store_id], 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 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 4 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [s_store_name@0 ASC NULLS LAST, s_store_id@1 ASC NULLS LAST, sun_sales@2 ASC NULLS LAST, mon_sales@3 ASC NULLS LAST, tue_sales@4 ASC NULLS LAST, wed_sales@5 ASC NULLS LAST, thu_sales@6 ASC NULLS LAST, fri_sales@7 ASC NULLS LAST, sat_sales@8 ASC NULLS LAST] + │ SortExec: TopK(fetch=100), expr=[s_store_name@0 ASC NULLS LAST, s_store_id@1 ASC NULLS LAST, sun_sales@2 ASC NULLS LAST, mon_sales@3 ASC NULLS LAST, tue_sales@4 ASC NULLS LAST, wed_sales@5 ASC NULLS LAST, thu_sales@6 ASC NULLS LAST, fri_sales@7 ASC NULLS LAST, sat_sales@8 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[s_store_name@0 as s_store_name, s_store_id@1 as s_store_id, 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=[s_store_name@0 as s_store_name, s_store_id@1 as s_store_id], 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 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([s_store_name@0, s_store_id@1], 6), input_partitions=2 @@ -4666,21 +4730,22 @@ mod tests { assert_snapshot!(display, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [rnk@0 ASC NULLS LAST], fetch=100 - │ [Stage 9] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 9] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[rnk@0 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[rnk@1 as rnk, i_product_name@2 as best_performing, i_product_name@0 as worst_performing] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, item_sk@1)], projection=[i_product_name@1, rnk@2, i_product_name@4] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=2 - │ ProjectionExec: expr=[rnk@1 as rnk, item_sk@2 as item_sk, i_product_name@0 as i_product_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, item_sk@0)], projection=[i_product_name@1, rnk@3, item_sk@4] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=2 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(rnk@1, rnk@1)], projection=[item_sk@0, rnk@1, item_sk@2] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=1 - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + ┌───── Stage 9 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [rnk@0 ASC NULLS LAST] + │ SortExec: TopK(fetch=100), expr=[rnk@0 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[rnk@1 as rnk, i_product_name@2 as best_performing, i_product_name@0 as worst_performing] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, item_sk@1)], projection=[i_product_name@1, rnk@2, i_product_name@4] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=2 + │ ProjectionExec: expr=[rnk@1 as rnk, item_sk@2 as item_sk, i_product_name@0 as i_product_name] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, item_sk@0)], projection=[i_product_name@1, rnk@3, item_sk@4] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=2 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(rnk@1, rnk@1)], projection=[item_sk@0, rnk@1, item_sk@2] + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=1 + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=1 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 @@ -4774,18 +4839,19 @@ mod tests { │ FilterExec: substr(ca_zip@2, 1, 5) IN (SET) ([85669, 86197, 88274, 83405, 86475, 85392, 85460, 80348, 81792]) OR mark@3, projection=[ws_sales_price@0, ca_city@1, ca_zip@2] │ HashJoinExec: mode=CollectLeft, join_type=LeftMark, on=[(i_item_id@3, i_item_id@0)], projection=[ws_sales_price@0, ca_city@1, ca_zip@2, mark@4] │ CoalescePartitionsExec - │ [Stage 6] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ FilterExec: i_item_sk@0 IN (SET) ([2, 3, 5, 7, 11, 13, 17, 19, 23, 29]), projection=[i_item_id@1] │ 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=i_item_sk@0 IN (SET) ([2, 3, 5, 7, 11, 13, 17, 19, 23, 29]), pruning_predicate=i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 2 AND 2 <= i_item_sk_max@1 OR i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 3 AND 3 <= i_item_sk_max@1 OR i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 5 AND 5 <= i_item_sk_max@1 OR i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 7 AND 7 <= i_item_sk_max@1 OR i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 11 AND 11 <= i_item_sk_max@1 OR i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 13 AND 13 <= i_item_sk_max@1 OR i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 17 AND 17 <= i_item_sk_max@1 OR i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 19 AND 19 <= i_item_sk_max@1 OR i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 23 AND 23 <= i_item_sk_max@1 OR i_item_sk_null_count@2 != row_count@3 AND i_item_sk_min@0 <= 29 AND 29 <= i_item_sk_max@1, required_guarantees=[i_item_sk in (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_item_sk@0, i_item_sk@0)], projection=[ws_sales_price@1, ca_city@2, ca_zip@3, i_item_id@5] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ 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, predicate=DynamicFilter [ empty ] + ┌───── Stage 6 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_item_sk@0, i_item_sk@0)], projection=[ws_sales_price@1, ca_city@2, ca_zip@3, i_item_id@5] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ 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, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -4835,17 +4901,18 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c_last_name@0 ASC, c_first_name@1 ASC, ca_city@2 ASC, bought_city@3 ASC, ss_ticket_number@4 ASC], fetch=100 - │ [Stage 8] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 8] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] - │ SortExec: TopK(fetch=100), expr=[c_last_name@0 ASC, c_first_name@1 ASC, ca_city@2 ASC, bought_city@3 ASC, ss_ticket_number@4 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[c_last_name@5 as c_last_name, c_first_name@4 as c_first_name, ca_city@6 as ca_city, bought_city@1 as bought_city, ss_ticket_number@0 as ss_ticket_number, amt@2 as amt, profit@3 as profit] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@4, ca_address_sk@0)], filter=bought_city@0 != ca_city@1, projection=[ss_ticket_number@0, bought_city@1, amt@2, profit@3, c_first_name@5, c_last_name@6, ca_city@8] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ 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_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], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 8 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [c_last_name@0 ASC, c_first_name@1 ASC, ca_city@2 ASC, bought_city@3 ASC, ss_ticket_number@4 ASC] + │ SortExec: TopK(fetch=100), expr=[c_last_name@0 ASC, c_first_name@1 ASC, ca_city@2 ASC, bought_city@3 ASC, ss_ticket_number@4 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[c_last_name@5 as c_last_name, c_first_name@4 as c_first_name, ca_city@6 as ca_city, bought_city@1 as bought_city, ss_ticket_number@0 as ss_ticket_number, amt@2 as amt, profit@3 as profit] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@4, ca_address_sk@0)], filter=bought_city@0 != ca_city@1, projection=[ss_ticket_number@0, bought_city@1, amt@2, profit@3, c_first_name@5, c_last_name@6, ca_city@8] + │ CoalescePartitionsExec + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ 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_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], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p6..p11] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 @@ -5077,18 +5144,19 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ AggregateExec: mode=Final, gby=[], aggr=[sum(store_sales.ss_quantity)] │ CoalescePartitionsExec - │ [Stage 5] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ [Stage 5] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ AggregateExec: mode=Partial, gby=[], aggr=[sum(store_sales.ss_quantity)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@1)], projection=[ss_quantity@1] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ 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)] - │ FilterExec: d_year@1 = 2000, projection=[d_date_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ 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@6 = 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 5 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(store_sales.ss_quantity)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@1)], projection=[ss_quantity@1] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ 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)] + │ FilterExec: d_year@1 = 2000, projection=[d_date_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ 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@6 = 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 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -5162,7 +5230,7 @@ mod tests { │ SortExec: expr=[currency_ratio@2 ASC NULLS LAST], preserve_partitioning=[false] │ BoundedWindowAggExec: wdw=[rank() ORDER BY [in_cat.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() ORDER BY [in_cat.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] │ SortPreservingMergeExec: [return_ratio@1 ASC NULLS LAST] - │ [Stage 9] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 9] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 │ ProjectionExec: expr=[store as channel, 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] │ FilterExec: rank() ORDER BY [in_store.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 <= 10 OR rank() ORDER BY [in_store.currency_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 <= 10 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1, maintains_sort_order=true @@ -5171,7 +5239,7 @@ mod tests { │ SortExec: expr=[currency_ratio@2 ASC NULLS LAST], preserve_partitioning=[false] │ BoundedWindowAggExec: wdw=[rank() ORDER BY [in_store.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() ORDER BY [in_store.return_ratio ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] │ SortPreservingMergeExec: [return_ratio@1 ASC NULLS LAST] - │ [Stage 14] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 14] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ RepartitionExec: partitioning=Hash([ws_item_sk@0], 3), input_partitions=3 @@ -5205,11 +5273,12 @@ mod tests { │ 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@33 > Some(100),7,2 AND ws_net_paid@29 > Some(0),7,2 AND ws_quantity@18 > 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 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: expr=[return_ratio@1 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[cs_item_sk@0 as item, CAST(sum(coalesce(cr.cr_return_quantity,Int64(0)))@1 AS Decimal128(15, 4)) / CAST(sum(coalesce(cs.cs_quantity,Int64(0)))@2 AS Decimal128(15, 4)) as return_ratio, CAST(sum(coalesce(cr.cr_return_amount,Int64(0)))@3 AS Decimal128(15, 4)) / CAST(sum(coalesce(cs.cs_net_paid,Int64(0)))@4 AS Decimal128(15, 4)) as currency_ratio] - │ AggregateExec: mode=FinalPartitioned, gby=[cs_item_sk@0 as cs_item_sk], aggr=[sum(coalesce(cr.cr_return_quantity,Int64(0))), sum(coalesce(cs.cs_quantity,Int64(0))), sum(coalesce(cr.cr_return_amount,Int64(0))), sum(coalesce(cs.cs_net_paid,Int64(0)))] - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 9 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [return_ratio@1 ASC NULLS LAST] + │ SortExec: expr=[return_ratio@1 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[cs_item_sk@0 as item, CAST(sum(coalesce(cr.cr_return_quantity,Int64(0)))@1 AS Decimal128(15, 4)) / CAST(sum(coalesce(cs.cs_quantity,Int64(0)))@2 AS Decimal128(15, 4)) as return_ratio, CAST(sum(coalesce(cr.cr_return_amount,Int64(0)))@3 AS Decimal128(15, 4)) / CAST(sum(coalesce(cs.cs_net_paid,Int64(0)))@4 AS Decimal128(15, 4)) as currency_ratio] + │ AggregateExec: mode=FinalPartitioned, gby=[cs_item_sk@0 as cs_item_sk], aggr=[sum(coalesce(cr.cr_return_quantity,Int64(0))), sum(coalesce(cs.cs_quantity,Int64(0))), sum(coalesce(cr.cr_return_amount,Int64(0))), sum(coalesce(cs.cs_net_paid,Int64(0)))] + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([cs_item_sk@0], 6), input_partitions=3 @@ -5243,11 +5312,12 @@ mod tests { │ 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@33 > Some(100),7,2 AND cs_net_paid@29 > Some(0),7,2 AND cs_quantity@18 > 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 14 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: expr=[return_ratio@1 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[ss_item_sk@0 as item, CAST(sum(coalesce(sr.sr_return_quantity,Int64(0)))@1 AS Decimal128(15, 4)) / CAST(sum(coalesce(sts.ss_quantity,Int64(0)))@2 AS Decimal128(15, 4)) as return_ratio, CAST(sum(coalesce(sr.sr_return_amt,Int64(0)))@3 AS Decimal128(15, 4)) / CAST(sum(coalesce(sts.ss_net_paid,Int64(0)))@4 AS Decimal128(15, 4)) as currency_ratio] - │ AggregateExec: mode=FinalPartitioned, gby=[ss_item_sk@0 as ss_item_sk], aggr=[sum(coalesce(sr.sr_return_quantity,Int64(0))), sum(coalesce(sts.ss_quantity,Int64(0))), sum(coalesce(sr.sr_return_amt,Int64(0))), sum(coalesce(sts.ss_net_paid,Int64(0)))] - │ [Stage 13] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 14 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [return_ratio@1 ASC NULLS LAST] + │ SortExec: expr=[return_ratio@1 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[ss_item_sk@0 as item, CAST(sum(coalesce(sr.sr_return_quantity,Int64(0)))@1 AS Decimal128(15, 4)) / CAST(sum(coalesce(sts.ss_quantity,Int64(0)))@2 AS Decimal128(15, 4)) as return_ratio, CAST(sum(coalesce(sr.sr_return_amt,Int64(0)))@3 AS Decimal128(15, 4)) / CAST(sum(coalesce(sts.ss_net_paid,Int64(0)))@4 AS Decimal128(15, 4)) as currency_ratio] + │ AggregateExec: mode=FinalPartitioned, gby=[ss_item_sk@0 as ss_item_sk], aggr=[sum(coalesce(sr.sr_return_quantity,Int64(0))), sum(coalesce(sts.ss_quantity,Int64(0))), sum(coalesce(sr.sr_return_amt,Int64(0))), sum(coalesce(sts.ss_net_paid,Int64(0)))] + │ [Stage 13] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 13 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([ss_item_sk@0], 6), input_partitions=3 @@ -5290,13 +5360,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [s_store_name@0 ASC NULLS LAST, s_company_id@1 ASC NULLS LAST, s_street_number@2 ASC NULLS LAST, s_street_name@3 ASC NULLS LAST, s_street_type@4 ASC NULLS LAST, s_suite_number@5 ASC NULLS LAST, s_city@6 ASC NULLS LAST, s_county@7 ASC NULLS LAST, s_state@8 ASC NULLS LAST, s_zip@9 ASC NULLS LAST], fetch=100 - │ [Stage 7] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 7] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[s_store_name@0 ASC NULLS LAST, s_company_id@1 ASC NULLS LAST, s_street_number@2 ASC NULLS LAST, s_street_name@3 ASC NULLS LAST, s_street_type@4 ASC NULLS LAST, s_suite_number@5 ASC NULLS LAST, s_city@6 ASC NULLS LAST, s_county@7 ASC NULLS LAST, s_state@8 ASC NULLS LAST, s_zip@9 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[s_store_name@0 as s_store_name, s_company_id@1 as s_company_id, s_street_number@2 as s_street_number, s_street_name@3 as s_street_name, s_street_type@4 as s_street_type, s_suite_number@5 as s_suite_number, s_city@6 as s_city, s_county@7 as s_county, s_state@8 as s_state, s_zip@9 as s_zip, sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END)@10 as 30 days, sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(30) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END)@11 as 31-60 days, sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(60) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END)@12 as 61-90 days, sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(90) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END)@13 as 91-120 days, sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)@14 as >120 days] - │ AggregateExec: mode=FinalPartitioned, gby=[s_store_name@0 as s_store_name, s_company_id@1 as s_company_id, s_street_number@2 as s_street_number, s_street_name@3 as s_street_name, s_street_type@4 as s_street_type, s_suite_number@5 as s_suite_number, s_city@6 as s_city, s_county@7 as s_county, s_state@8 as s_state, s_zip@9 as s_zip], aggr=[sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(30) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(60) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(90) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)] - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 7 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [s_store_name@0 ASC NULLS LAST, s_company_id@1 ASC NULLS LAST, s_street_number@2 ASC NULLS LAST, s_street_name@3 ASC NULLS LAST, s_street_type@4 ASC NULLS LAST, s_suite_number@5 ASC NULLS LAST, s_city@6 ASC NULLS LAST, s_county@7 ASC NULLS LAST, s_state@8 ASC NULLS LAST, s_zip@9 ASC NULLS LAST] + │ SortExec: TopK(fetch=100), expr=[s_store_name@0 ASC NULLS LAST, s_company_id@1 ASC NULLS LAST, s_street_number@2 ASC NULLS LAST, s_street_name@3 ASC NULLS LAST, s_street_type@4 ASC NULLS LAST, s_suite_number@5 ASC NULLS LAST, s_city@6 ASC NULLS LAST, s_county@7 ASC NULLS LAST, s_state@8 ASC NULLS LAST, s_zip@9 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[s_store_name@0 as s_store_name, s_company_id@1 as s_company_id, s_street_number@2 as s_street_number, s_street_name@3 as s_street_name, s_street_type@4 as s_street_type, s_suite_number@5 as s_suite_number, s_city@6 as s_city, s_county@7 as s_county, s_state@8 as s_state, s_zip@9 as s_zip, sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END)@10 as 30 days, sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(30) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END)@11 as 31-60 days, sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(60) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END)@12 as 61-90 days, sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(90) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END)@13 as 91-120 days, sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)@14 as >120 days] + │ AggregateExec: mode=FinalPartitioned, gby=[s_store_name@0 as s_store_name, s_company_id@1 as s_company_id, s_street_number@2 as s_street_number, s_street_name@3 as s_street_name, s_street_type@4 as s_street_type, s_suite_number@5 as s_suite_number, s_city@6 as s_city, s_county@7 as s_county, s_state@8 as s_state, s_zip@9 as s_zip], aggr=[sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(30) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(60) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(90) AND store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN store_returns.sr_returned_date_sk - store_sales.ss_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)] + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([s_store_name@0, s_company_id@1, s_street_number@2, s_street_name@3, s_street_type@4, s_suite_number@5, s_city@6, s_county@7, s_state@8, s_zip@9], 6), input_partitions=3 @@ -5431,13 +5502,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [d_year@0 ASC NULLS LAST, ext_price@3 DESC, brand_id@1 ASC NULLS LAST], fetch=100 - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[ext_price@3 DESC, brand_id@1 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[d_year@0 as d_year, i_brand_id@2 as brand_id, i_brand@1 as brand, sum(store_sales.ss_ext_sales_price)@3 as ext_price] - │ AggregateExec: mode=FinalPartitioned, gby=[d_year@0 as d_year, i_brand@1 as i_brand, i_brand_id@2 as i_brand_id], aggr=[sum(store_sales.ss_ext_sales_price)], ordering_mode=PartiallySorted([0]) - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 4 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [d_year@0 ASC NULLS LAST, ext_price@3 DESC, brand_id@1 ASC NULLS LAST] + │ SortExec: TopK(fetch=100), expr=[ext_price@3 DESC, brand_id@1 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[d_year@0 as d_year, i_brand_id@2 as brand_id, i_brand@1 as brand, sum(store_sales.ss_ext_sales_price)@3 as ext_price] + │ AggregateExec: mode=FinalPartitioned, gby=[d_year@0 as d_year, i_brand@1 as i_brand, i_brand_id@2 as i_brand_id], aggr=[sum(store_sales.ss_ext_sales_price)], ordering_mode=PartiallySorted([0]) + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([d_year@0, i_brand@1, i_brand_id@2], 6), input_partitions=3 @@ -5655,13 +5727,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [ext_price@2 DESC, brand_id@0 ASC NULLS LAST], fetch=100 - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[ext_price@2 DESC, brand_id@0 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[i_brand_id@1 as brand_id, i_brand@0 as brand, sum(store_sales.ss_ext_sales_price)@2 as ext_price] - │ AggregateExec: mode=FinalPartitioned, gby=[i_brand@0 as i_brand, i_brand_id@1 as i_brand_id], aggr=[sum(store_sales.ss_ext_sales_price)] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 4 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [ext_price@2 DESC, brand_id@0 ASC NULLS LAST] + │ SortExec: TopK(fetch=100), expr=[ext_price@2 DESC, brand_id@0 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[i_brand_id@1 as brand_id, i_brand@0 as brand, sum(store_sales.ss_ext_sales_price)@2 as ext_price] + │ AggregateExec: mode=FinalPartitioned, gby=[i_brand@0 as i_brand, i_brand_id@1 as i_brand_id], aggr=[sum(store_sales.ss_ext_sales_price)] + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([i_brand@0, i_brand_id@1], 6), input_partitions=3 @@ -5709,7 +5782,7 @@ mod tests { │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(store_sales.ss_ext_sales_price)] │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(i_item_id@1, i_item_id@0)] │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ FilterExec: i_color@1 = slate OR i_color@1 = blanched OR i_color@1 = burnished, projection=[i_item_id@0] │ 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_id, i_color], file_type=parquet, predicate=i_color@17 = slate OR i_color@17 = blanched OR i_color@17 = burnished, pruning_predicate=i_color_null_count@2 != row_count@3 AND i_color_min@0 <= slate AND slate <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= blanched AND blanched <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= burnished AND burnished <= i_color_max@1, required_guarantees=[i_color in (blanched, burnished, slate)] @@ -5719,7 +5792,7 @@ mod tests { │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(catalog_sales.cs_ext_sales_price)] │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(i_item_id@1, i_item_id@0)] │ CoalescePartitionsExec - │ [Stage 8] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ [Stage 8] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ FilterExec: i_color@1 = slate OR i_color@1 = blanched OR i_color@1 = burnished, projection=[i_item_id@0] │ 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_id, i_color], file_type=parquet, predicate=i_color@17 = slate OR i_color@17 = blanched OR i_color@17 = burnished, pruning_predicate=i_color_null_count@2 != row_count@3 AND i_color_min@0 <= slate AND slate <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= blanched AND blanched <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= burnished AND burnished <= i_color_max@1, required_guarantees=[i_color in (blanched, burnished, slate)] @@ -5729,18 +5802,19 @@ mod tests { │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(web_sales.ws_ext_sales_price)] │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(i_item_id@1, i_item_id@0)] │ CoalescePartitionsExec - │ [Stage 12] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ [Stage 12] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ FilterExec: i_color@1 = slate OR i_color@1 = blanched OR i_color@1 = burnished, projection=[i_item_id@0] │ 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_id, i_color], file_type=parquet, predicate=i_color@17 = slate OR i_color@17 = blanched OR i_color@17 = burnished, pruning_predicate=i_color_null_count@2 != row_count@3 AND i_color_min@0 <= slate AND slate <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= blanched AND blanched <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= burnished AND burnished <= i_color_max@1, required_guarantees=[i_color in (blanched, burnished, slate)] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_ext_sales_price@1, i_item_id@3] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ 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, predicate=DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_ext_sales_price@1, i_item_id@3] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ 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, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -5769,13 +5843,14 @@ 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@6 = 2001 AND d_moy@8 = 2, 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 <= 2 AND 2 <= d_moy_max@5, required_guarantees=[d_moy in (2), d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, i_item_sk@0)], projection=[cs_ext_sales_price@1, i_item_id@3] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ 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, predicate=DynamicFilter [ empty ] + ┌───── Stage 8 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, i_item_sk@0)], projection=[cs_ext_sales_price@1, i_item_id@3] + │ CoalescePartitionsExec + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ 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, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -5804,13 +5879,14 @@ 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@6 = 2001 AND d_moy@8 = 2, 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 <= 2 AND 2 <= d_moy_max@5, required_guarantees=[d_moy in (2), d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_item_sk@0, i_item_sk@0)], projection=[ws_ext_sales_price@1, i_item_id@3] - │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ 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, predicate=DynamicFilter [ empty ] + ┌───── Stage 12 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_item_sk@0, i_item_sk@0)], projection=[ws_ext_sales_price@1, i_item_id@3] + │ CoalescePartitionsExec + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ 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, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -6015,7 +6091,7 @@ mod tests { │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(web_sales.ws_ext_sales_price)] │ 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 4] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_week_seq@0, d_week_seq@1)], projection=[d_date@1] │ CoalescePartitionsExec │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 @@ -6034,7 +6110,7 @@ mod tests { │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(catalog_sales.cs_ext_sales_price)] │ 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 8] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ [Stage 8] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_week_seq@0, d_week_seq@1)], projection=[d_date@1] │ CoalescePartitionsExec │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 @@ -6049,7 +6125,7 @@ mod tests { │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(store_sales.ss_ext_sales_price)] │ 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 12] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ [Stage 12] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_week_seq@0, d_week_seq@1)], projection=[d_date@1] │ CoalescePartitionsExec │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 @@ -6059,11 +6135,12 @@ 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 4 ── 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] - │ 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 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 4 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ 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] + │ 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 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p0..p8] │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@2], 9), input_partitions=3 @@ -6086,11 +6163,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_item_id], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 8 ── 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] - │ 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 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 8 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ 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] + │ 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 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p0..p8] │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@2], 9), input_partitions=3 @@ -6113,11 +6191,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_item_id], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 12 ── 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] - │ 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 9] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 12 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ 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] + │ 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 9] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p0..p8] │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@2], 9), input_partitions=3 @@ -6149,22 +6228,23 @@ mod tests { assert_snapshot!(display, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [s_store_name1@0 ASC, s_store_id1@1 ASC, d_week_seq1@2 ASC], fetch=100 - │ [Stage 10] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 10] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p3..p5] - │ SortExec: TopK(fetch=100), expr=[s_store_name1@0 ASC, s_store_id1@1 ASC, d_week_seq1@2 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[s_store_name1@0 as s_store_name1, s_store_id1@2 as s_store_id1, d_week_seq1@1 as d_week_seq1, sun_sales1@3 / sun_sales2@10 as sun_sales_ratio, mon_sales1@4 / mon_sales2@11 as mon_sales_ratio, tue_sales1@5 / tue_sales2@12 as tue_sales_ratio, wed_sales1@6 / wed_sales2@13 as wed_sales_ratio, thu_sales1@7 / thu_sales2@14 as thu_sales_ratio, fri_sales1@8 / fri_sales2@15 as fri_sales_ratio, sat_sales1@9 / sat_sales2@16 as sat_sales_ratio] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_store_id1@2, s_store_id2@1), (CAST(y.d_week_seq1 AS Int64)@10, x.d_week_seq2 - Int64(52)@9)], projection=[s_store_name1@0, d_week_seq1@1, s_store_id1@2, sun_sales1@3, mon_sales1@4, tue_sales1@5, wed_sales1@6, thu_sales1@7, fri_sales1@8, sat_sales1@9, sun_sales2@13, mon_sales2@14, tue_sales2@15, wed_sales2@16, thu_sales2@17, fri_sales2@18, sat_sales2@19] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[d_week_seq@0 as d_week_seq2, s_store_id@8 as s_store_id2, sun_sales@1 as sun_sales2, mon_sales@2 as mon_sales2, tue_sales@3 as tue_sales2, wed_sales@4 as wed_sales2, thu_sales@5 as thu_sales2, fri_sales@6 as fri_sales2, sat_sales@7 as sat_sales2, CAST(d_week_seq@0 AS Int64) - 52 as x.d_week_seq2 - Int64(52)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_week_seq@0, d_week_seq@0)], projection=[d_week_seq@0, sun_sales@1, mon_sales@2, tue_sales@3, wed_sales@4, thu_sales@5, fri_sales@6, sat_sales@7, s_store_id@8] - │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ FilterExec: d_month_seq@0 >= 1224 AND d_month_seq@0 <= 1235, projection=[d_week_seq@1] - │ 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_month_seq, d_week_seq], file_type=parquet, predicate=d_month_seq@3 >= 1224 AND d_month_seq@3 <= 1235 AND DynamicFilter [ empty ], pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1224 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1235, required_guarantees=[] + ┌───── Stage 10 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [s_store_name1@0 ASC, s_store_id1@1 ASC, d_week_seq1@2 ASC] + │ SortExec: TopK(fetch=100), expr=[s_store_name1@0 ASC, s_store_id1@1 ASC, d_week_seq1@2 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[s_store_name1@0 as s_store_name1, s_store_id1@2 as s_store_id1, d_week_seq1@1 as d_week_seq1, sun_sales1@3 / sun_sales2@10 as sun_sales_ratio, mon_sales1@4 / mon_sales2@11 as mon_sales_ratio, tue_sales1@5 / tue_sales2@12 as tue_sales_ratio, wed_sales1@6 / wed_sales2@13 as wed_sales_ratio, thu_sales1@7 / thu_sales2@14 as thu_sales_ratio, fri_sales1@8 / fri_sales2@15 as fri_sales_ratio, sat_sales1@9 / sat_sales2@16 as sat_sales_ratio] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_store_id1@2, s_store_id2@1), (CAST(y.d_week_seq1 AS Int64)@10, x.d_week_seq2 - Int64(52)@9)], projection=[s_store_name1@0, d_week_seq1@1, s_store_id1@2, sun_sales1@3, mon_sales1@4, tue_sales1@5, wed_sales1@6, thu_sales1@7, fri_sales1@8, sat_sales1@9, sun_sales2@13, mon_sales2@14, tue_sales2@15, wed_sales2@16, thu_sales2@17, fri_sales2@18, sat_sales2@19] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ ProjectionExec: expr=[d_week_seq@0 as d_week_seq2, s_store_id@8 as s_store_id2, sun_sales@1 as sun_sales2, mon_sales@2 as mon_sales2, tue_sales@3 as tue_sales2, wed_sales@4 as wed_sales2, thu_sales@5 as thu_sales2, fri_sales@6 as fri_sales2, sat_sales@7 as sat_sales2, CAST(d_week_seq@0 AS Int64) - 52 as x.d_week_seq2 - Int64(52)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_week_seq@0, d_week_seq@0)], projection=[d_week_seq@0, sun_sales@1, mon_sales@2, tue_sales@3, wed_sales@4, thu_sales@5, fri_sales@6, sat_sales@7, s_store_id@8] + │ CoalescePartitionsExec + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ FilterExec: d_month_seq@0 >= 1224 AND d_month_seq@0 <= 1235, projection=[d_week_seq@1] + │ 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_month_seq, d_week_seq], file_type=parquet, predicate=d_month_seq@3 >= 1224 AND d_month_seq@3 <= 1235 AND DynamicFilter [ empty ], pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1224 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1235, required_guarantees=[] └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 @@ -6260,7 +6340,7 @@ mod tests { │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(store_sales.ss_ext_sales_price)] │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(i_item_id@1, i_item_id@0)] │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ FilterExec: i_category@1 = Music, projection=[i_item_id@0] │ 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_id, i_category], file_type=parquet, predicate=i_category@12 = Music, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Music AND Music <= i_category_max@1, required_guarantees=[i_category in (Music)] @@ -6270,7 +6350,7 @@ mod tests { │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(catalog_sales.cs_ext_sales_price)] │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(i_item_id@1, i_item_id@0)] │ CoalescePartitionsExec - │ [Stage 8] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ [Stage 8] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ FilterExec: i_category@1 = Music, projection=[i_item_id@0] │ 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_id, i_category], file_type=parquet, predicate=i_category@12 = Music, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Music AND Music <= i_category_max@1, required_guarantees=[i_category in (Music)] @@ -6280,18 +6360,19 @@ mod tests { │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(web_sales.ws_ext_sales_price)] │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(i_item_id@1, i_item_id@0)] │ CoalescePartitionsExec - │ [Stage 12] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ [Stage 12] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ FilterExec: i_category@1 = Music, projection=[i_item_id@0] │ 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_id, i_category], file_type=parquet, predicate=i_category@12 = Music, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Music AND Music <= i_category_max@1, required_guarantees=[i_category in (Music)] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_ext_sales_price@1, i_item_id@3] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ 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, predicate=DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_ext_sales_price@1, i_item_id@3] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ 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, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -6320,13 +6401,14 @@ 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@6 = 1998 AND d_moy@8 = 9, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 9 AND 9 <= d_moy_max@5, required_guarantees=[d_moy in (9), d_year in (1998)] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, i_item_sk@0)], projection=[cs_ext_sales_price@1, i_item_id@3] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ 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, predicate=DynamicFilter [ empty ] + ┌───── Stage 8 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, i_item_sk@0)], projection=[cs_ext_sales_price@1, i_item_id@3] + │ CoalescePartitionsExec + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ 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, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -6355,13 +6437,14 @@ 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@6 = 1998 AND d_moy@8 = 9, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 9 AND 9 <= d_moy_max@5, required_guarantees=[d_moy in (9), d_year in (1998)] └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_item_sk@0, i_item_sk@0)], projection=[ws_ext_sales_price@1, i_item_id@3] - │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ 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, predicate=DynamicFilter [ empty ] + ┌───── Stage 12 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_item_sk@0, i_item_sk@0)], projection=[ws_ext_sales_price@1, i_item_id@3] + │ CoalescePartitionsExec + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ 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, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -6404,21 +6487,22 @@ mod tests { │ ProjectionExec: expr=[sum(store_sales.ss_ext_sales_price)@0 as promotions] │ AggregateExec: mode=Final, gby=[], aggr=[sum(store_sales.ss_ext_sales_price)] │ CoalescePartitionsExec - │ [Stage 7] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ [Stage 7] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[sum(store_sales.ss_ext_sales_price)@0 as total] │ AggregateExec: mode=Final, gby=[], aggr=[sum(store_sales.ss_ext_sales_price)] │ CoalescePartitionsExec - │ [Stage 13] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ [Stage 13] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ AggregateExec: mode=Partial, gby=[], aggr=[sum(store_sales.ss_ext_sales_price)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_ext_sales_price@1] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ FilterExec: i_category@1 = Jewelry, projection=[i_item_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ 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, predicate=i_category@12 = Jewelry AND DynamicFilter [ empty ], pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Jewelry AND Jewelry <= i_category_max@1, required_guarantees=[i_category in (Jewelry)] + ┌───── Stage 7 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(store_sales.ss_ext_sales_price)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_ext_sales_price@1] + │ CoalescePartitionsExec + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ FilterExec: i_category@1 = Jewelry, projection=[i_item_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ 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, predicate=i_category@12 = Jewelry AND DynamicFilter [ empty ], pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Jewelry AND Jewelry <= i_category_max@1, required_guarantees=[i_category in (Jewelry)] └────────────────────────────────────────────────── ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -6478,15 +6562,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_gmt_offset], file_type=parquet, predicate=s_gmt_offset@27 = Some(-500),3,2, pruning_predicate=s_gmt_offset_null_count@2 != row_count@3 AND s_gmt_offset_min@0 <= Some(-500),3,2 AND Some(-500),3,2 <= s_gmt_offset_max@1, required_guarantees=[s_gmt_offset in (Some(-500),3,2)] └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ AggregateExec: mode=Partial, gby=[], aggr=[sum(store_sales.ss_ext_sales_price)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_ext_sales_price@1] - │ CoalescePartitionsExec - │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ FilterExec: i_category@1 = Jewelry, projection=[i_item_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ 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, predicate=i_category@12 = Jewelry AND DynamicFilter [ empty ], pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Jewelry AND Jewelry <= i_category_max@1, required_guarantees=[i_category in (Jewelry)] + ┌───── Stage 13 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(store_sales.ss_ext_sales_price)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_ext_sales_price@1] + │ CoalescePartitionsExec + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ FilterExec: i_category@1 = Jewelry, projection=[i_item_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ 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, predicate=i_category@12 = Jewelry AND DynamicFilter [ empty ], pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Jewelry AND Jewelry <= i_category_max@1, required_guarantees=[i_category in (Jewelry)] └────────────────────────────────────────────────── ┌───── Stage 12 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -6544,13 +6629,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [w_substr@0 ASC, sm_type@1 ASC, web_name@2 ASC], fetch=100 - │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[w_substr@0 ASC, sm_type@1 ASC, web_name@2 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[w_substr@0 as w_substr, sm_type@1 as sm_type, web_name@2 as web_name, sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END)@3 as 30 days, sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(30) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END)@4 as 31-60 days, sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(60) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END)@5 as 61-90 days, sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(90) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END)@6 as 91-120 days, sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)@7 as >120 days] - │ AggregateExec: mode=FinalPartitioned, gby=[w_substr@0 as w_substr, sm_type@1 as sm_type, web_name@2 as web_name], aggr=[sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(30) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(60) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(90) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 6 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [w_substr@0 ASC, sm_type@1 ASC, web_name@2 ASC] + │ SortExec: TopK(fetch=100), expr=[w_substr@0 ASC, sm_type@1 ASC, web_name@2 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[w_substr@0 as w_substr, sm_type@1 as sm_type, web_name@2 as web_name, sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END)@3 as 30 days, sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(30) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END)@4 as 31-60 days, sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(60) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END)@5 as 61-90 days, sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(90) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END)@6 as 91-120 days, sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)@7 as >120 days] + │ AggregateExec: mode=FinalPartitioned, gby=[w_substr@0 as w_substr, sm_type@1 as sm_type, web_name@2 as web_name], aggr=[sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(30) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(60) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(90) AND web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN web_sales.ws_ship_date_sk - web_sales.ws_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)] + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([w_substr@0, sm_type@1, web_name@2], 6), input_partitions=3 @@ -6663,17 +6749,18 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[product_name@0 as product_name, store_name@1 as store_name, store_zip@2 as store_zip, b_street_number@3 as b_street_number, b_street_name@4 as b_street_name, b_city@5 as b_city, b_zip@6 as b_zip, c_street_number@7 as c_street_number, c_street_name@8 as c_street_name, c_city@9 as c_city, c_zip@10 as c_zip, cs1syear@11 as cs1syear, cs1cnt@12 as cs1cnt, s11@13 as s11, s21@14 as s21, s31@15 as s31, s12@16 as s12, s22@17 as s22, s32@18 as s32, syear@19 as syear, cnt@20 as cnt] │ SortPreservingMergeExec: [product_name@0 ASC NULLS LAST, store_name@1 ASC NULLS LAST, cnt@20 ASC NULLS LAST, s1@21 ASC NULLS LAST, s1@22 ASC NULLS LAST] - │ [Stage 46] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 46] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 46 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: expr=[product_name@0 ASC NULLS LAST, store_name@1 ASC NULLS LAST, cnt@20 ASC NULLS LAST, s11@13 ASC NULLS LAST, s12@16 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[product_name@0 as product_name, store_name@1 as store_name, store_zip@2 as store_zip, b_street_number@3 as b_street_number, b_street_name@4 as b_street_name, b_city@5 as b_city, b_zip@6 as b_zip, c_street_number@7 as c_street_number, c_street_name@8 as c_street_name, c_city@9 as c_city, c_zip@10 as c_zip, syear@11 as cs1syear, cnt@12 as cs1cnt, s1@13 as s11, s2@14 as s21, s3@15 as s31, s1@18 as s12, s2@19 as s22, s3@20 as s32, syear@16 as syear, cnt@17 as cnt, s1@13 as s1, s1@18 as s1] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(item_sk@1, item_sk@0), (store_name@2, store_name@1), (store_zip@3, store_zip@2)], filter=cnt@1 <= cnt@0, projection=[product_name@0, store_name@2, store_zip@3, b_street_number@4, b_street_name@5, b_city@6, b_zip@7, c_street_number@8, c_street_name@9, c_city@10, c_zip@11, syear@12, cnt@13, s1@14, s2@15, s3@16, syear@20, cnt@21, s1@22, s2@23, s3@24] - │ CoalescePartitionsExec - │ [Stage 23] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[i_item_sk@1 as item_sk, s_store_name@2 as store_name, s_zip@3 as store_zip, d_year@12 as syear, count(Int64(1))@15 as cnt, sum(store_sales.ss_wholesale_cost)@16 as s1, sum(store_sales.ss_list_price)@17 as s2, sum(store_sales.ss_coupon_amt)@18 as s3] - │ AggregateExec: mode=FinalPartitioned, gby=[i_product_name@0 as i_product_name, i_item_sk@1 as i_item_sk, s_store_name@2 as s_store_name, s_zip@3 as s_zip, ca_street_number@4 as ca_street_number, ca_street_name@5 as ca_street_name, ca_city@6 as ca_city, ca_zip@7 as ca_zip, ca_street_number@8 as ca_street_number, ca_street_name@9 as ca_street_name, ca_city@10 as ca_city, ca_zip@11 as ca_zip, d_year@12 as d_year, d_year@13 as d_year, d_year@14 as d_year], aggr=[count(Int64(1)), sum(store_sales.ss_wholesale_cost), sum(store_sales.ss_list_price), sum(store_sales.ss_coupon_amt)], ordering_mode=PartiallySorted([12]) - │ [Stage 45] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 46 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [product_name@0 ASC NULLS LAST, store_name@1 ASC NULLS LAST, cnt@20 ASC NULLS LAST, s1@21 ASC NULLS LAST, s1@22 ASC NULLS LAST] + │ SortExec: expr=[product_name@0 ASC NULLS LAST, store_name@1 ASC NULLS LAST, cnt@20 ASC NULLS LAST, s11@13 ASC NULLS LAST, s12@16 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[product_name@0 as product_name, store_name@1 as store_name, store_zip@2 as store_zip, b_street_number@3 as b_street_number, b_street_name@4 as b_street_name, b_city@5 as b_city, b_zip@6 as b_zip, c_street_number@7 as c_street_number, c_street_name@8 as c_street_name, c_city@9 as c_city, c_zip@10 as c_zip, syear@11 as cs1syear, cnt@12 as cs1cnt, s1@13 as s11, s2@14 as s21, s3@15 as s31, s1@18 as s12, s2@19 as s22, s3@20 as s32, syear@16 as syear, cnt@17 as cnt, s1@13 as s1, s1@18 as s1] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(item_sk@1, item_sk@0), (store_name@2, store_name@1), (store_zip@3, store_zip@2)], filter=cnt@1 <= cnt@0, projection=[product_name@0, store_name@2, store_zip@3, b_street_number@4, b_street_name@5, b_city@6, b_zip@7, c_street_number@8, c_street_name@9, c_city@10, c_zip@11, syear@12, cnt@13, s1@14, s2@15, s3@16, syear@20, cnt@21, s1@22, s2@23, s3@24] + │ CoalescePartitionsExec + │ [Stage 23] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ ProjectionExec: expr=[i_item_sk@1 as item_sk, s_store_name@2 as store_name, s_zip@3 as store_zip, d_year@12 as syear, count(Int64(1))@15 as cnt, sum(store_sales.ss_wholesale_cost)@16 as s1, sum(store_sales.ss_list_price)@17 as s2, sum(store_sales.ss_coupon_amt)@18 as s3] + │ AggregateExec: mode=FinalPartitioned, gby=[i_product_name@0 as i_product_name, i_item_sk@1 as i_item_sk, s_store_name@2 as s_store_name, s_zip@3 as s_zip, ca_street_number@4 as ca_street_number, ca_street_name@5 as ca_street_name, ca_city@6 as ca_city, ca_zip@7 as ca_zip, ca_street_number@8 as ca_street_number, ca_street_name@9 as ca_street_name, ca_city@10 as ca_city, ca_zip@11 as ca_zip, d_year@12 as d_year, d_year@13 as d_year, d_year@14 as d_year], aggr=[count(Int64(1)), sum(store_sales.ss_wholesale_cost), sum(store_sales.ss_list_price), sum(store_sales.ss_coupon_amt)], ordering_mode=PartiallySorted([12]) + │ [Stage 45] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 23 ── Tasks: t0:[p0..p5] t1:[p6..p11] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 @@ -7052,17 +7139,18 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [s_store_name@0 ASC, i_item_desc@1 ASC], fetch=100 - │ [Stage 9] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 9] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[s_store_name@0 ASC, i_item_desc@1 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[s_store_name@0 as s_store_name, i_item_desc@2 as i_item_desc, revenue@1 as revenue, i_current_price@3 as i_current_price, i_wholesale_cost@4 as i_wholesale_cost, i_brand@5 as i_brand] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_store_sk@1, ss_store_sk@0)], filter=CAST(revenue@0 AS Decimal128(30, 15)) <= CAST(0.1 * CAST(ave@1 AS Float64) AS Decimal128(30, 15)), projection=[s_store_name@0, revenue@2, i_item_desc@3, i_current_price@4, i_wholesale_cost@5, i_brand@6] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[ss_store_sk@0 as ss_store_sk, avg(sa.revenue)@1 as ave] - │ AggregateExec: mode=FinalPartitioned, gby=[ss_store_sk@0 as ss_store_sk], aggr=[avg(sa.revenue)] - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + ┌───── Stage 9 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [s_store_name@0 ASC, i_item_desc@1 ASC] + │ SortExec: TopK(fetch=100), expr=[s_store_name@0 ASC, i_item_desc@1 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[s_store_name@0 as s_store_name, i_item_desc@2 as i_item_desc, revenue@1 as revenue, i_current_price@3 as i_current_price, i_wholesale_cost@4 as i_wholesale_cost, i_brand@5 as i_brand] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_store_sk@1, ss_store_sk@0)], filter=CAST(revenue@0 AS Decimal128(30, 15)) <= CAST(0.1 * CAST(ave@1 AS Float64) AS Decimal128(30, 15)), projection=[s_store_name@0, revenue@2, i_item_desc@3, i_current_price@4, i_wholesale_cost@5, i_brand@6] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ ProjectionExec: expr=[ss_store_sk@0 as ss_store_sk, avg(sa.revenue)@1 as ave] + │ AggregateExec: mode=FinalPartitioned, gby=[ss_store_sk@0 as ss_store_sk], aggr=[avg(sa.revenue)] + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=2 └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 @@ -7320,17 +7408,18 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c_last_name@0 ASC, ss_ticket_number@4 ASC], fetch=100 - │ [Stage 8] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 8] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] - │ SortExec: TopK(fetch=100), expr=[c_last_name@0 ASC, ss_ticket_number@4 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[c_last_name@6 as c_last_name, c_first_name@5 as c_first_name, ca_city@7 as ca_city, bought_city@1 as bought_city, ss_ticket_number@0 as ss_ticket_number, extended_price@2 as extended_price, extended_tax@4 as extended_tax, list_price@3 as list_price] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@5, ca_address_sk@0)], filter=bought_city@0 != ca_city@1, projection=[ss_ticket_number@0, bought_city@1, extended_price@2, list_price@3, extended_tax@4, c_first_name@6, c_last_name@7, ca_city@9] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ 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_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], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 8 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [c_last_name@0 ASC, ss_ticket_number@4 ASC] + │ SortExec: TopK(fetch=100), expr=[c_last_name@0 ASC, ss_ticket_number@4 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[c_last_name@6 as c_last_name, c_first_name@5 as c_first_name, ca_city@7 as ca_city, bought_city@1 as bought_city, ss_ticket_number@0 as ss_ticket_number, extended_price@2 as extended_price, extended_tax@4 as extended_tax, list_price@3 as list_price] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@5, ca_address_sk@0)], filter=bought_city@0 != ca_city@1, projection=[ss_ticket_number@0, bought_city@1, extended_price@2, list_price@3, extended_tax@4, c_first_name@6, c_last_name@7, ca_city@9] + │ CoalescePartitionsExec + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ 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_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], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p6..p11] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 @@ -7406,13 +7495,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [cd_gender@0 ASC NULLS LAST, cd_marital_status@1 ASC NULLS LAST, cd_education_status@2 ASC NULLS LAST, cd_purchase_estimate@4 ASC NULLS LAST, cd_credit_rating@6 ASC NULLS LAST], fetch=100 - │ [Stage 10] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 10] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[cd_gender@0 ASC NULLS LAST, cd_marital_status@1 ASC NULLS LAST, cd_education_status@2 ASC NULLS LAST, cd_purchase_estimate@4 ASC NULLS LAST, cd_credit_rating@6 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[cd_gender@0 as cd_gender, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, count(Int64(1))@5 as cnt1, cd_purchase_estimate@3 as cd_purchase_estimate, count(Int64(1))@5 as cnt2, cd_credit_rating@4 as cd_credit_rating, count(Int64(1))@5 as cnt3] - │ AggregateExec: mode=FinalPartitioned, gby=[cd_gender@0 as cd_gender, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, cd_purchase_estimate@3 as cd_purchase_estimate, cd_credit_rating@4 as cd_credit_rating], aggr=[count(Int64(1))] - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 10 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [cd_gender@0 ASC NULLS LAST, cd_marital_status@1 ASC NULLS LAST, cd_education_status@2 ASC NULLS LAST, cd_purchase_estimate@4 ASC NULLS LAST, cd_credit_rating@6 ASC NULLS LAST] + │ SortExec: TopK(fetch=100), expr=[cd_gender@0 ASC NULLS LAST, cd_marital_status@1 ASC NULLS LAST, cd_education_status@2 ASC NULLS LAST, cd_purchase_estimate@4 ASC NULLS LAST, cd_credit_rating@6 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[cd_gender@0 as cd_gender, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, count(Int64(1))@5 as cnt1, cd_purchase_estimate@3 as cd_purchase_estimate, count(Int64(1))@5 as cnt2, cd_credit_rating@4 as cd_credit_rating, count(Int64(1))@5 as cnt3] + │ AggregateExec: mode=FinalPartitioned, gby=[cd_gender@0 as cd_gender, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, cd_purchase_estimate@3 as cd_purchase_estimate, cd_credit_rating@4 as cd_credit_rating], aggr=[count(Int64(1))] + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([cd_gender@0, cd_marital_status@1, cd_education_status@2, cd_purchase_estimate@3, cd_credit_rating@4], 6), input_partitions=2 @@ -7515,13 +7605,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [ext_price@4 DESC, brand_id@0 ASC, t_hour@2 ASC] - │ [Stage 5] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 5] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: expr=[ext_price@4 DESC, brand_id@0 ASC, t_hour@2 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[i_brand_id@1 as brand_id, i_brand@0 as brand, t_hour@2 as t_hour, t_minute@3 as t_minute, sum(tmp.ext_price)@4 as ext_price] - │ AggregateExec: mode=FinalPartitioned, gby=[i_brand@0 as i_brand, i_brand_id@1 as i_brand_id, t_hour@2 as t_hour, t_minute@3 as t_minute], aggr=[sum(tmp.ext_price)] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + ┌───── Stage 5 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [ext_price@4 DESC, brand_id@0 ASC, t_hour@2 ASC] + │ SortExec: expr=[ext_price@4 DESC, brand_id@0 ASC, t_hour@2 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[i_brand_id@1 as brand_id, i_brand@0 as brand, t_hour@2 as t_hour, t_minute@3 as t_minute, sum(tmp.ext_price)@4 as ext_price] + │ AggregateExec: mode=FinalPartitioned, gby=[i_brand@0 as i_brand, i_brand_id@1 as i_brand_id, t_hour@2 as t_hour, t_minute@3 as t_minute], aggr=[sum(tmp.ext_price)] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ RepartitionExec: partitioning=Hash([i_brand@0, i_brand_id@1, t_hour@2, t_minute@3], 6), input_partitions=3 @@ -7599,21 +7690,22 @@ mod tests { │ AggregateExec: mode=Partial, gby=[i_item_desc@1 as i_item_desc, w_warehouse_name@0 as w_warehouse_name, d_week_seq@2 as d_week_seq], aggr=[sum(CASE WHEN promotion.p_promo_sk IS NULL THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN promotion.p_promo_sk IS NOT NULL THEN Int64(1) ELSE Int64(0) END), count(Int64(1))] │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(cs_item_sk@0, cr_item_sk@0), (cs_order_number@1, cr_order_number@1)], projection=[w_warehouse_name@2, i_item_desc@3, d_week_seq@4, p_promo_sk@5] │ CoalescePartitionsExec - │ [Stage 12] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 + │ [Stage 12] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ DataSourceExec: file_groups={3 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:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-3.parquet:..]]}, projection=[cr_item_sk, cr_order_number], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] - │ ProjectionExec: expr=[cs_item_sk@1 as cs_item_sk, cs_order_number@2 as cs_order_number, w_warehouse_name@3 as w_warehouse_name, i_item_desc@4 as i_item_desc, d_week_seq@5 as d_week_seq, p_promo_sk@0 as p_promo_sk] - │ HashJoinExec: mode=CollectLeft, join_type=Right, on=[(CAST(promotion.p_promo_sk AS Float64)@1, cs_promo_sk@1)], projection=[p_promo_sk@0, cs_item_sk@2, cs_order_number@4, w_warehouse_name@5, i_item_desc@6, d_week_seq@7] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_ship_date_sk@0, CAST(d3.d_date_sk AS Float64)@2)], filter=d_date@1 > d_date@0 + IntervalMonthDayNano { months: 0, days: 5, nanoseconds: 0 }, projection=[cs_item_sk@1, cs_promo_sk@2, cs_order_number@3, w_warehouse_name@4, i_item_desc@5, d_week_seq@7] + ┌───── Stage 12 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ ProjectionExec: expr=[cs_item_sk@1 as cs_item_sk, cs_order_number@2 as cs_order_number, w_warehouse_name@3 as w_warehouse_name, i_item_desc@4 as i_item_desc, d_week_seq@5 as d_week_seq, p_promo_sk@0 as p_promo_sk] + │ HashJoinExec: mode=CollectLeft, join_type=Right, on=[(CAST(promotion.p_promo_sk AS Float64)@1, cs_promo_sk@1)], projection=[p_promo_sk@0, cs_item_sk@2, cs_order_number@4, w_warehouse_name@5, i_item_desc@6, d_week_seq@7] │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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(d3.d_date_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ 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 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_ship_date_sk@0, CAST(d3.d_date_sk AS Float64)@2)], filter=d_date@1 > d_date@0 + IntervalMonthDayNano { months: 0, days: 5, nanoseconds: 0 }, projection=[cs_item_sk@1, cs_promo_sk@2, cs_order_number@3, w_warehouse_name@4, i_item_desc@5, d_week_seq@7] + │ CoalescePartitionsExec + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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(d3.d_date_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ 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 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 @@ -7708,18 +7800,19 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [cnt@5 DESC, c_last_name@0 ASC NULLS LAST] - │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] - │ SortExec: expr=[cnt@5 DESC, c_last_name@0 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[c_last_name@4 as c_last_name, c_first_name@3 as c_first_name, c_salutation@2 as c_salutation, c_preferred_cust_flag@5 as c_preferred_cust_flag, ss_ticket_number@0 as ss_ticket_number, cnt@1 as cnt] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@5)], projection=[ss_ticket_number@0, cnt@2, c_salutation@4, c_first_name@5, c_last_name@6, c_preferred_cust_flag@7] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_salutation@1 as c_salutation, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_preferred_cust_flag@4 as c_preferred_cust_flag, 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_salutation, c_first_name, c_last_name, c_preferred_cust_flag], file_type=parquet + ┌───── Stage 6 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [cnt@5 DESC, c_last_name@0 ASC NULLS LAST] + │ SortExec: expr=[cnt@5 DESC, c_last_name@0 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[c_last_name@4 as c_last_name, c_first_name@3 as c_first_name, c_salutation@2 as c_salutation, c_preferred_cust_flag@5 as c_preferred_cust_flag, ss_ticket_number@0 as ss_ticket_number, cnt@1 as cnt] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@5)], projection=[ss_ticket_number@0, cnt@2, c_salutation@4, c_first_name@5, c_last_name@6, c_preferred_cust_flag@7] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_salutation@1 as c_salutation, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_preferred_cust_flag@4 as c_preferred_cust_flag, 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_salutation, c_first_name, c_last_name, c_preferred_cust_flag], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 @@ -7776,16 +7869,17 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [customer_id@0 ASC], fetch=100 - │ [Stage 16] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 16] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[customer_id@0 ASC], preserve_partitioning=[true] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)], filter=CASE WHEN year_total@2 > Some(0),17,2 THEN year_total@3 / year_total@2 END > CASE WHEN year_total@0 > Some(0),17,2 THEN year_total@1 / year_total@0 END, projection=[customer_id@2, customer_first_name@3, customer_last_name@4] - │ CoalescePartitionsExec - │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ 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]) - │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 16 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [customer_id@0 ASC] + │ SortExec: TopK(fetch=100), expr=[customer_id@0 ASC], preserve_partitioning=[true] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)], filter=CASE WHEN year_total@2 > Some(0),17,2 THEN year_total@3 / year_total@2 END > CASE WHEN year_total@0 > Some(0),17,2 THEN year_total@1 / year_total@0 END, projection=[customer_id@2, customer_first_name@3, customer_last_name@4] + │ CoalescePartitionsExec + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ 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]) + │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 12 ── Tasks: t0:[p0..p5] t1:[p6..p11] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 @@ -7963,33 +8057,34 @@ mod tests { │ ProjectionExec: expr=[CAST(cr_return_amount@8 AS Decimal128(30, 15)) as __common_expr_1, cs_quantity@0 as cs_quantity, cs_ext_sales_price@1 as cs_ext_sales_price, i_brand_id@2 as i_brand_id, i_class_id@3 as i_class_id, i_category_id@4 as i_category_id, i_manufact_id@5 as i_manufact_id, d_year@6 as d_year, cr_return_quantity@7 as cr_return_quantity] │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(cs_order_number@1, cr_order_number@1), (cs_item_sk@0, cr_item_sk@0)], projection=[cs_quantity@2, cs_ext_sales_price@3, i_brand_id@4, i_class_id@5, i_category_id@6, i_manufact_id@7, d_year@8, cr_return_quantity@11, cr_return_amount@12] │ CoalescePartitionsExec - │ [Stage 3] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 3] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ DataSourceExec: file_groups={3 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:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-3.parquet:..]]}, projection=[cr_item_sk, cr_order_number, cr_return_quantity, cr_return_amount], file_type=parquet │ ProjectionExec: expr=[d_year@7 as d_year, i_brand_id@3 as i_brand_id, i_class_id@4 as i_class_id, i_category_id@5 as i_category_id, i_manufact_id@6 as i_manufact_id, ss_quantity@1 - CASE WHEN sr_return_quantity@8 IS NOT NULL THEN sr_return_quantity@8 ELSE 0 END as sales_cnt, ss_ext_sales_price@2 - CASE WHEN __common_expr_2@0 IS NOT NULL THEN __common_expr_2@0 ELSE Some(0),30,15 END as sales_amt] │ ProjectionExec: expr=[CAST(sr_return_amt@8 AS Decimal128(30, 15)) as __common_expr_2, ss_quantity@0 as ss_quantity, ss_ext_sales_price@1 as ss_ext_sales_price, i_brand_id@2 as i_brand_id, i_class_id@3 as i_class_id, i_category_id@4 as i_category_id, i_manufact_id@5 as i_manufact_id, d_year@6 as d_year, sr_return_quantity@7 as sr_return_quantity] │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(ss_ticket_number@1, sr_ticket_number@1), (ss_item_sk@0, sr_item_sk@0)], projection=[ss_quantity@2, ss_ext_sales_price@3, i_brand_id@4, i_class_id@5, i_category_id@6, i_manufact_id@7, d_year@8, sr_return_quantity@11, sr_return_amt@12] │ CoalescePartitionsExec - │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ DataSourceExec: file_groups={3 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:.., /testdata/tpcds/plans_sf1_partitions4/store_returns/part-3.parquet:..]]}, projection=[sr_item_sk, sr_ticket_number, sr_return_quantity, sr_return_amt], file_type=parquet │ ProjectionExec: expr=[d_year@7 as d_year, i_brand_id@3 as i_brand_id, i_class_id@4 as i_class_id, i_category_id@5 as i_category_id, i_manufact_id@6 as i_manufact_id, ws_quantity@1 - CASE WHEN wr_return_quantity@8 IS NOT NULL THEN wr_return_quantity@8 ELSE 0 END as sales_cnt, ws_ext_sales_price@2 - CASE WHEN __common_expr_3@0 IS NOT NULL THEN __common_expr_3@0 ELSE Some(0),30,15 END as sales_amt] │ ProjectionExec: expr=[CAST(wr_return_amt@8 AS Decimal128(30, 15)) as __common_expr_3, ws_quantity@0 as ws_quantity, ws_ext_sales_price@1 as ws_ext_sales_price, i_brand_id@2 as i_brand_id, i_class_id@3 as i_class_id, i_category_id@4 as i_category_id, i_manufact_id@5 as i_manufact_id, d_year@6 as d_year, wr_return_quantity@7 as wr_return_quantity] │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(ws_order_number@1, wr_order_number@1), (ws_item_sk@0, wr_item_sk@0)], projection=[ws_quantity@2, ws_ext_sales_price@3, i_brand_id@4, i_class_id@5, i_category_id@6, i_manufact_id@7, d_year@8, wr_return_quantity@11, wr_return_amt@12] │ CoalescePartitionsExec - │ [Stage 9] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 9] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 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_quantity, wr_return_amt], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ ProjectionExec: expr=[cs_item_sk@1 as cs_item_sk, cs_order_number@2 as cs_order_number, cs_quantity@3 as cs_quantity, cs_ext_sales_price@4 as cs_ext_sales_price, i_brand_id@5 as i_brand_id, i_class_id@6 as i_class_id, i_category_id@7 as i_category_id, i_manufact_id@8 as i_manufact_id, d_year@0 as d_year] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, cs_sold_date_sk@0)], projection=[d_year@1, cs_item_sk@4, cs_order_number@5, cs_quantity@6, cs_ext_sales_price@7, i_brand_id@8, i_class_id@9, i_category_id@10, i_manufact_id@11] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[cs_sold_date_sk@4 as cs_sold_date_sk, cs_item_sk@5 as cs_item_sk, cs_order_number@6 as cs_order_number, cs_quantity@7 as cs_quantity, cs_ext_sales_price@8 as cs_ext_sales_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, i_manufact_id@3 as i_manufact_id] - │ HashJoinExec: mode=CollectLeft, 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, i_manufact_id@4, cs_sold_date_sk@5, cs_item_sk@6, cs_order_number@7, cs_quantity@8, cs_ext_sales_price@9] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_order_number, cs_quantity, cs_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 3 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ ProjectionExec: expr=[cs_item_sk@1 as cs_item_sk, cs_order_number@2 as cs_order_number, cs_quantity@3 as cs_quantity, cs_ext_sales_price@4 as cs_ext_sales_price, i_brand_id@5 as i_brand_id, i_class_id@6 as i_class_id, i_category_id@7 as i_category_id, i_manufact_id@8 as i_manufact_id, d_year@0 as d_year] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, cs_sold_date_sk@0)], projection=[d_year@1, cs_item_sk@4, cs_order_number@5, cs_quantity@6, cs_ext_sales_price@7, i_brand_id@8, i_class_id@9, i_category_id@10, i_manufact_id@11] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ ProjectionExec: expr=[cs_sold_date_sk@4 as cs_sold_date_sk, cs_item_sk@5 as cs_item_sk, cs_order_number@6 as cs_order_number, cs_quantity@7 as cs_quantity, cs_ext_sales_price@8 as cs_ext_sales_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, i_manufact_id@3 as i_manufact_id] + │ HashJoinExec: mode=CollectLeft, 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, i_manufact_id@4, cs_sold_date_sk@5, cs_item_sk@6, cs_order_number@7, cs_quantity@8, cs_ext_sales_price@9] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_order_number, cs_quantity, cs_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -8006,17 +8101,18 @@ 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, i_category, i_manufact_id], file_type=parquet, predicate=i_category@12 = Books, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1, required_guarantees=[i_category in (Books)] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_ticket_number@2 as ss_ticket_number, ss_quantity@3 as ss_quantity, ss_ext_sales_price@4 as ss_ext_sales_price, i_brand_id@5 as i_brand_id, i_class_id@6 as i_class_id, i_category_id@7 as i_category_id, i_manufact_id@8 as i_manufact_id, d_year@0 as d_year] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_ticket_number@5, ss_quantity@6, ss_ext_sales_price@7, i_brand_id@8, i_class_id@9, i_category_id@10, i_manufact_id@11] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ss_sold_date_sk@4 as ss_sold_date_sk, ss_item_sk@5 as ss_item_sk, ss_ticket_number@6 as ss_ticket_number, ss_quantity@7 as ss_quantity, ss_ext_sales_price@8 as ss_ext_sales_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, i_manufact_id@3 as i_manufact_id] - │ HashJoinExec: mode=CollectLeft, 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, i_manufact_id@4, ss_sold_date_sk@5, ss_item_sk@6, ss_ticket_number@7, ss_quantity@8, ss_ext_sales_price@9] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_ticket_number, ss_quantity, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 6 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_ticket_number@2 as ss_ticket_number, ss_quantity@3 as ss_quantity, ss_ext_sales_price@4 as ss_ext_sales_price, i_brand_id@5 as i_brand_id, i_class_id@6 as i_class_id, i_category_id@7 as i_category_id, i_manufact_id@8 as i_manufact_id, d_year@0 as d_year] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_ticket_number@5, ss_quantity@6, ss_ext_sales_price@7, i_brand_id@8, i_class_id@9, i_category_id@10, i_manufact_id@11] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ ProjectionExec: expr=[ss_sold_date_sk@4 as ss_sold_date_sk, ss_item_sk@5 as ss_item_sk, ss_ticket_number@6 as ss_ticket_number, ss_quantity@7 as ss_quantity, ss_ext_sales_price@8 as ss_ext_sales_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, i_manufact_id@3 as i_manufact_id] + │ HashJoinExec: mode=CollectLeft, 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, i_manufact_id@4, ss_sold_date_sk@5, ss_item_sk@6, ss_ticket_number@7, ss_quantity@8, ss_ext_sales_price@9] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_ticket_number, ss_quantity, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -8033,17 +8129,18 @@ 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, i_category, i_manufact_id], file_type=parquet, predicate=i_category@12 = Books, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1, required_guarantees=[i_category in (Books)] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ ProjectionExec: expr=[ws_item_sk@1 as ws_item_sk, ws_order_number@2 as ws_order_number, ws_quantity@3 as ws_quantity, ws_ext_sales_price@4 as ws_ext_sales_price, i_brand_id@5 as i_brand_id, i_class_id@6 as i_class_id, i_category_id@7 as i_category_id, i_manufact_id@8 as i_manufact_id, d_year@0 as d_year] - │ HashJoinExec: mode=CollectLeft, 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_order_number@5, ws_quantity@6, ws_ext_sales_price@7, i_brand_id@8, i_class_id@9, i_category_id@10, i_manufact_id@11] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ws_sold_date_sk@4 as ws_sold_date_sk, ws_item_sk@5 as ws_item_sk, ws_order_number@6 as ws_order_number, ws_quantity@7 as ws_quantity, ws_ext_sales_price@8 as ws_ext_sales_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, i_manufact_id@3 as i_manufact_id] - │ HashJoinExec: mode=CollectLeft, 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, i_manufact_id@4, ws_sold_date_sk@5, ws_item_sk@6, ws_order_number@7, ws_quantity@8, ws_ext_sales_price@9] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_order_number, ws_quantity, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 9 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ws_item_sk@1 as ws_item_sk, ws_order_number@2 as ws_order_number, ws_quantity@3 as ws_quantity, ws_ext_sales_price@4 as ws_ext_sales_price, i_brand_id@5 as i_brand_id, i_class_id@6 as i_class_id, i_category_id@7 as i_category_id, i_manufact_id@8 as i_manufact_id, d_year@0 as d_year] + │ HashJoinExec: mode=CollectLeft, 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_order_number@5, ws_quantity@6, ws_ext_sales_price@7, i_brand_id@8, i_class_id@9, i_category_id@10, i_manufact_id@11] + │ CoalescePartitionsExec + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ ProjectionExec: expr=[ws_sold_date_sk@4 as ws_sold_date_sk, ws_item_sk@5 as ws_item_sk, ws_order_number@6 as ws_order_number, ws_quantity@7 as ws_quantity, ws_ext_sales_price@8 as ws_ext_sales_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, i_manufact_id@3 as i_manufact_id] + │ HashJoinExec: mode=CollectLeft, 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, i_manufact_id@4, ws_sold_date_sk@5, ws_item_sk@6, ws_order_number@7, ws_quantity@8, ws_ext_sales_price@9] + │ CoalescePartitionsExec + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_order_number, ws_quantity, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p9..p17] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -8074,33 +8171,34 @@ mod tests { │ ProjectionExec: expr=[CAST(cr_return_amount@8 AS Decimal128(30, 15)) as __common_expr_4, cs_quantity@0 as cs_quantity, cs_ext_sales_price@1 as cs_ext_sales_price, i_brand_id@2 as i_brand_id, i_class_id@3 as i_class_id, i_category_id@4 as i_category_id, i_manufact_id@5 as i_manufact_id, d_year@6 as d_year, cr_return_quantity@7 as cr_return_quantity] │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(cs_order_number@1, cr_order_number@1), (cs_item_sk@0, cr_item_sk@0)], projection=[cs_quantity@2, cs_ext_sales_price@3, i_brand_id@4, i_class_id@5, i_category_id@6, i_manufact_id@7, d_year@8, cr_return_quantity@11, cr_return_amount@12] │ CoalescePartitionsExec - │ [Stage 14] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 14] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ DataSourceExec: file_groups={3 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:.., /testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-3.parquet:..]]}, projection=[cr_item_sk, cr_order_number, cr_return_quantity, cr_return_amount], file_type=parquet │ ProjectionExec: expr=[d_year@7 as d_year, i_brand_id@3 as i_brand_id, i_class_id@4 as i_class_id, i_category_id@5 as i_category_id, i_manufact_id@6 as i_manufact_id, ss_quantity@1 - CASE WHEN sr_return_quantity@8 IS NOT NULL THEN sr_return_quantity@8 ELSE 0 END as sales_cnt, ss_ext_sales_price@2 - CASE WHEN __common_expr_5@0 IS NOT NULL THEN __common_expr_5@0 ELSE Some(0),30,15 END as sales_amt] │ ProjectionExec: expr=[CAST(sr_return_amt@8 AS Decimal128(30, 15)) as __common_expr_5, ss_quantity@0 as ss_quantity, ss_ext_sales_price@1 as ss_ext_sales_price, i_brand_id@2 as i_brand_id, i_class_id@3 as i_class_id, i_category_id@4 as i_category_id, i_manufact_id@5 as i_manufact_id, d_year@6 as d_year, sr_return_quantity@7 as sr_return_quantity] │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(ss_ticket_number@1, sr_ticket_number@1), (ss_item_sk@0, sr_item_sk@0)], projection=[ss_quantity@2, ss_ext_sales_price@3, i_brand_id@4, i_class_id@5, i_category_id@6, i_manufact_id@7, d_year@8, sr_return_quantity@11, sr_return_amt@12] │ CoalescePartitionsExec - │ [Stage 17] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 17] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ DataSourceExec: file_groups={3 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:.., /testdata/tpcds/plans_sf1_partitions4/store_returns/part-3.parquet:..]]}, projection=[sr_item_sk, sr_ticket_number, sr_return_quantity, sr_return_amt], file_type=parquet │ ProjectionExec: expr=[d_year@7 as d_year, i_brand_id@3 as i_brand_id, i_class_id@4 as i_class_id, i_category_id@5 as i_category_id, i_manufact_id@6 as i_manufact_id, ws_quantity@1 - CASE WHEN wr_return_quantity@8 IS NOT NULL THEN wr_return_quantity@8 ELSE 0 END as sales_cnt, ws_ext_sales_price@2 - CASE WHEN __common_expr_6@0 IS NOT NULL THEN __common_expr_6@0 ELSE Some(0),30,15 END as sales_amt] │ ProjectionExec: expr=[CAST(wr_return_amt@8 AS Decimal128(30, 15)) as __common_expr_6, ws_quantity@0 as ws_quantity, ws_ext_sales_price@1 as ws_ext_sales_price, i_brand_id@2 as i_brand_id, i_class_id@3 as i_class_id, i_category_id@4 as i_category_id, i_manufact_id@5 as i_manufact_id, d_year@6 as d_year, wr_return_quantity@7 as wr_return_quantity] │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(ws_order_number@1, wr_order_number@1), (ws_item_sk@0, wr_item_sk@0)], projection=[ws_quantity@2, ws_ext_sales_price@3, i_brand_id@4, i_class_id@5, i_category_id@6, i_manufact_id@7, d_year@8, wr_return_quantity@11, wr_return_amt@12] │ CoalescePartitionsExec - │ [Stage 20] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 20] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 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_quantity, wr_return_amt], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ ProjectionExec: expr=[cs_item_sk@1 as cs_item_sk, cs_order_number@2 as cs_order_number, cs_quantity@3 as cs_quantity, cs_ext_sales_price@4 as cs_ext_sales_price, i_brand_id@5 as i_brand_id, i_class_id@6 as i_class_id, i_category_id@7 as i_category_id, i_manufact_id@8 as i_manufact_id, d_year@0 as d_year] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, cs_sold_date_sk@0)], projection=[d_year@1, cs_item_sk@4, cs_order_number@5, cs_quantity@6, cs_ext_sales_price@7, i_brand_id@8, i_class_id@9, i_category_id@10, i_manufact_id@11] - │ CoalescePartitionsExec - │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[cs_sold_date_sk@4 as cs_sold_date_sk, cs_item_sk@5 as cs_item_sk, cs_order_number@6 as cs_order_number, cs_quantity@7 as cs_quantity, cs_ext_sales_price@8 as cs_ext_sales_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, i_manufact_id@3 as i_manufact_id] - │ HashJoinExec: mode=CollectLeft, 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, i_manufact_id@4, cs_sold_date_sk@5, cs_item_sk@6, cs_order_number@7, cs_quantity@8, cs_ext_sales_price@9] - │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_order_number, cs_quantity, cs_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 14 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ ProjectionExec: expr=[cs_item_sk@1 as cs_item_sk, cs_order_number@2 as cs_order_number, cs_quantity@3 as cs_quantity, cs_ext_sales_price@4 as cs_ext_sales_price, i_brand_id@5 as i_brand_id, i_class_id@6 as i_class_id, i_category_id@7 as i_category_id, i_manufact_id@8 as i_manufact_id, d_year@0 as d_year] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, cs_sold_date_sk@0)], projection=[d_year@1, cs_item_sk@4, cs_order_number@5, cs_quantity@6, cs_ext_sales_price@7, i_brand_id@8, i_class_id@9, i_category_id@10, i_manufact_id@11] + │ CoalescePartitionsExec + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ ProjectionExec: expr=[cs_sold_date_sk@4 as cs_sold_date_sk, cs_item_sk@5 as cs_item_sk, cs_order_number@6 as cs_order_number, cs_quantity@7 as cs_quantity, cs_ext_sales_price@8 as cs_ext_sales_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, i_manufact_id@3 as i_manufact_id] + │ HashJoinExec: mode=CollectLeft, 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, i_manufact_id@4, cs_sold_date_sk@5, cs_item_sk@6, cs_order_number@7, cs_quantity@8, cs_ext_sales_price@9] + │ CoalescePartitionsExec + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_order_number, cs_quantity, cs_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 12 ── Tasks: t0:[p0..p8] t1:[p9..p17] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -8117,17 +8215,18 @@ 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, i_category, i_manufact_id], file_type=parquet, predicate=i_category@12 = Books, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1, required_guarantees=[i_category in (Books)] └────────────────────────────────────────────────── - ┌───── Stage 17 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_ticket_number@2 as ss_ticket_number, ss_quantity@3 as ss_quantity, ss_ext_sales_price@4 as ss_ext_sales_price, i_brand_id@5 as i_brand_id, i_class_id@6 as i_class_id, i_category_id@7 as i_category_id, i_manufact_id@8 as i_manufact_id, d_year@0 as d_year] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_ticket_number@5, ss_quantity@6, ss_ext_sales_price@7, i_brand_id@8, i_class_id@9, i_category_id@10, i_manufact_id@11] - │ CoalescePartitionsExec - │ [Stage 15] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ss_sold_date_sk@4 as ss_sold_date_sk, ss_item_sk@5 as ss_item_sk, ss_ticket_number@6 as ss_ticket_number, ss_quantity@7 as ss_quantity, ss_ext_sales_price@8 as ss_ext_sales_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, i_manufact_id@3 as i_manufact_id] - │ HashJoinExec: mode=CollectLeft, 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, i_manufact_id@4, ss_sold_date_sk@5, ss_item_sk@6, ss_ticket_number@7, ss_quantity@8, ss_ext_sales_price@9] - │ CoalescePartitionsExec - │ [Stage 16] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_ticket_number, ss_quantity, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 17 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_ticket_number@2 as ss_ticket_number, ss_quantity@3 as ss_quantity, ss_ext_sales_price@4 as ss_ext_sales_price, i_brand_id@5 as i_brand_id, i_class_id@6 as i_class_id, i_category_id@7 as i_category_id, i_manufact_id@8 as i_manufact_id, d_year@0 as d_year] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_ticket_number@5, ss_quantity@6, ss_ext_sales_price@7, i_brand_id@8, i_class_id@9, i_category_id@10, i_manufact_id@11] + │ CoalescePartitionsExec + │ [Stage 15] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ ProjectionExec: expr=[ss_sold_date_sk@4 as ss_sold_date_sk, ss_item_sk@5 as ss_item_sk, ss_ticket_number@6 as ss_ticket_number, ss_quantity@7 as ss_quantity, ss_ext_sales_price@8 as ss_ext_sales_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, i_manufact_id@3 as i_manufact_id] + │ HashJoinExec: mode=CollectLeft, 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, i_manufact_id@4, ss_sold_date_sk@5, ss_item_sk@6, ss_ticket_number@7, ss_quantity@8, ss_ext_sales_price@9] + │ CoalescePartitionsExec + │ [Stage 16] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_ticket_number, ss_quantity, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 15 ── Tasks: t0:[p0..p8] t1:[p9..p17] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -8144,17 +8243,18 @@ 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, i_category, i_manufact_id], file_type=parquet, predicate=i_category@12 = Books, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1, required_guarantees=[i_category in (Books)] └────────────────────────────────────────────────── - ┌───── Stage 20 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ ProjectionExec: expr=[ws_item_sk@1 as ws_item_sk, ws_order_number@2 as ws_order_number, ws_quantity@3 as ws_quantity, ws_ext_sales_price@4 as ws_ext_sales_price, i_brand_id@5 as i_brand_id, i_class_id@6 as i_class_id, i_category_id@7 as i_category_id, i_manufact_id@8 as i_manufact_id, d_year@0 as d_year] - │ HashJoinExec: mode=CollectLeft, 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_order_number@5, ws_quantity@6, ws_ext_sales_price@7, i_brand_id@8, i_class_id@9, i_category_id@10, i_manufact_id@11] - │ CoalescePartitionsExec - │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ws_sold_date_sk@4 as ws_sold_date_sk, ws_item_sk@5 as ws_item_sk, ws_order_number@6 as ws_order_number, ws_quantity@7 as ws_quantity, ws_ext_sales_price@8 as ws_ext_sales_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, i_manufact_id@3 as i_manufact_id] - │ HashJoinExec: mode=CollectLeft, 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, i_manufact_id@4, ws_sold_date_sk@5, ws_item_sk@6, ws_order_number@7, ws_quantity@8, ws_ext_sales_price@9] - │ CoalescePartitionsExec - │ [Stage 19] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_order_number, ws_quantity, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 20 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ws_item_sk@1 as ws_item_sk, ws_order_number@2 as ws_order_number, ws_quantity@3 as ws_quantity, ws_ext_sales_price@4 as ws_ext_sales_price, i_brand_id@5 as i_brand_id, i_class_id@6 as i_class_id, i_category_id@7 as i_category_id, i_manufact_id@8 as i_manufact_id, d_year@0 as d_year] + │ HashJoinExec: mode=CollectLeft, 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_order_number@5, ws_quantity@6, ws_ext_sales_price@7, i_brand_id@8, i_class_id@9, i_category_id@10, i_manufact_id@11] + │ CoalescePartitionsExec + │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ ProjectionExec: expr=[ws_sold_date_sk@4 as ws_sold_date_sk, ws_item_sk@5 as ws_item_sk, ws_order_number@6 as ws_order_number, ws_quantity@7 as ws_quantity, ws_ext_sales_price@8 as ws_ext_sales_price, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, i_manufact_id@3 as i_manufact_id] + │ HashJoinExec: mode=CollectLeft, 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, i_manufact_id@4, ws_sold_date_sk@5, ws_item_sk@6, ws_order_number@7, ws_quantity@8, ws_ext_sales_price@9] + │ CoalescePartitionsExec + │ [Stage 19] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_order_number, ws_quantity, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 18 ── Tasks: t0:[p0..p8] t1:[p9..p17] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -8180,13 +8280,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [channel@0 ASC, col_name@1 ASC, d_year@2 ASC, d_qoy@3 ASC, i_category@4 ASC], fetch=100 - │ [Stage 11] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 11] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[channel@0 ASC, col_name@1 ASC, d_year@2 ASC, d_qoy@3 ASC, i_category@4 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[channel@0 as channel, col_name@1 as col_name, d_year@2 as d_year, d_qoy@3 as d_qoy, i_category@4 as i_category, count(Int64(1))@5 as sales_cnt, sum(foo.ext_sales_price)@6 as sales_amt] - │ AggregateExec: mode=FinalPartitioned, gby=[channel@0 as channel, col_name@1 as col_name, d_year@2 as d_year, d_qoy@3 as d_qoy, i_category@4 as i_category], aggr=[count(Int64(1)), sum(foo.ext_sales_price)] - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + ┌───── Stage 11 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [channel@0 ASC, col_name@1 ASC, d_year@2 ASC, d_qoy@3 ASC, i_category@4 ASC] + │ SortExec: TopK(fetch=100), expr=[channel@0 ASC, col_name@1 ASC, d_year@2 ASC, d_qoy@3 ASC, i_category@4 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[channel@0 as channel, col_name@1 as col_name, d_year@2 as d_year, d_qoy@3 as d_qoy, i_category@4 as i_category, count(Int64(1))@5 as sales_cnt, sum(foo.ext_sales_price)@6 as sales_amt] + │ AggregateExec: mode=FinalPartitioned, gby=[channel@0 as channel, col_name@1 as col_name, d_year@2 as d_year, d_qoy@3 as d_qoy, i_category@4 as i_category], aggr=[count(Int64(1)), sum(foo.ext_sales_price)] + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ RepartitionExec: partitioning=Hash([channel@0, col_name@1, d_year@2, d_qoy@3, i_category@4], 6), input_partitions=3 @@ -8280,13 +8381,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [channel@0 ASC, id@1 ASC, returns_@3 DESC], fetch=100 - │ [Stage 21] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 21] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 21 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[channel@0 ASC, id@1 ASC, returns_@3 DESC], preserve_partitioning=[true] - │ ProjectionExec: expr=[channel@0 as channel, id@1 as id, sum(x.sales)@3 as sales, sum(x.returns_)@4 as returns_, sum(x.profit)@5 as profit] - │ AggregateExec: mode=FinalPartitioned, gby=[channel@0 as channel, id@1 as id, __grouping_id@2 as __grouping_id], aggr=[sum(x.sales), sum(x.returns_), sum(x.profit)] - │ [Stage 20] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + ┌───── Stage 21 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [channel@0 ASC, id@1 ASC, returns_@3 DESC] + │ SortExec: TopK(fetch=100), expr=[channel@0 ASC, id@1 ASC, returns_@3 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[channel@0 as channel, id@1 as id, sum(x.sales)@3 as sales, sum(x.returns_)@4 as returns_, sum(x.profit)@5 as profit] + │ AggregateExec: mode=FinalPartitioned, gby=[channel@0 as channel, id@1 as id, __grouping_id@2 as __grouping_id], aggr=[sum(x.sales), sum(x.returns_), sum(x.profit)] + │ [Stage 20] => NetworkShuffleExec: output_partitions=3, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 20 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ RepartitionExec: partitioning=Hash([channel@0, id@1, __grouping_id@2], 6), input_partitions=3 @@ -8303,7 +8405,7 @@ mod tests { │ ProjectionExec: expr=[catalog channel as channel, cs_call_center_sk@0 as id, sales@1 as sales, CAST(returns_@3 AS Decimal128(22, 2)) as returns_, CAST(profit@2 - profit_loss@4 AS Decimal128(23, 2)) as profit] │ CrossJoinExec │ CoalescePartitionsExec - │ [Stage 10] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 10] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 │ ProjectionExec: expr=[sum(catalog_returns.cr_return_amount)@1 as returns_, sum(catalog_returns.cr_net_loss)@2 as profit_loss] │ AggregateExec: mode=FinalPartitioned, gby=[cr_call_center_sk@0 as cr_call_center_sk], aggr=[sum(catalog_returns.cr_return_amount), sum(catalog_returns.cr_net_loss)] │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=3 @@ -8311,7 +8413,7 @@ mod tests { │ ProjectionExec: expr=[CAST(returns_@3 AS Decimal128(22, 2)) as __common_expr_3, CAST(profit_loss@4 AS Decimal128(22, 2)) as __common_expr_4, wp_web_page_sk@0 as wp_web_page_sk, sales@1 as sales, profit@2 as profit] │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(wp_web_page_sk@0, wp_web_page_sk@0)], projection=[wp_web_page_sk@0, sales@1, profit@2, returns_@4, profit_loss@5] │ CoalescePartitionsExec - │ [Stage 16] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 16] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 │ ProjectionExec: expr=[wp_web_page_sk@0 as wp_web_page_sk, sum(web_returns.wr_return_amt)@1 as returns_, sum(web_returns.wr_net_loss)@2 as profit_loss] │ AggregateExec: mode=FinalPartitioned, gby=[wp_web_page_sk@0 as wp_web_page_sk], aggr=[sum(web_returns.wr_return_amt), sum(web_returns.wr_net_loss)] │ [Stage 19] => NetworkShuffleExec: output_partitions=3, input_tasks=2 @@ -8374,10 +8476,11 @@ 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@2 >= 2000-08-23 AND d_date@2 <= 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] - │ ProjectionExec: expr=[cs_call_center_sk@0 as cs_call_center_sk, sum(catalog_sales.cs_ext_sales_price)@1 as sales, sum(catalog_sales.cs_net_profit)@2 as profit] - │ AggregateExec: mode=FinalPartitioned, gby=[cs_call_center_sk@0 as cs_call_center_sk], aggr=[sum(catalog_sales.cs_ext_sales_price), sum(catalog_sales.cs_net_profit)] - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 10 ── Tasks: t0:[p0] t1:[p1] + │ CoalescePartitionsExec + │ ProjectionExec: expr=[cs_call_center_sk@0 as cs_call_center_sk, sum(catalog_sales.cs_ext_sales_price)@1 as sales, sum(catalog_sales.cs_net_profit)@2 as profit] + │ AggregateExec: mode=FinalPartitioned, gby=[cs_call_center_sk@0 as cs_call_center_sk], aggr=[sum(catalog_sales.cs_ext_sales_price), sum(catalog_sales.cs_net_profit)] + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([cs_call_center_sk@0], 6), input_partitions=2 @@ -8412,10 +8515,11 @@ 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@2 >= 2000-08-23 AND d_date@2 <= 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 16 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ ProjectionExec: expr=[wp_web_page_sk@0 as wp_web_page_sk, sum(web_sales.ws_ext_sales_price)@1 as sales, sum(web_sales.ws_net_profit)@2 as profit] - │ AggregateExec: mode=FinalPartitioned, gby=[wp_web_page_sk@0 as wp_web_page_sk], aggr=[sum(web_sales.ws_ext_sales_price), sum(web_sales.ws_net_profit)] - │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 16 ── Tasks: t0:[p0] t1:[p1] + │ CoalescePartitionsExec + │ ProjectionExec: expr=[wp_web_page_sk@0 as wp_web_page_sk, sum(web_sales.ws_ext_sales_price)@1 as sales, sum(web_sales.ws_net_profit)@2 as profit] + │ AggregateExec: mode=FinalPartitioned, gby=[wp_web_page_sk@0 as wp_web_page_sk], aggr=[sum(web_sales.ws_ext_sales_price), sum(web_sales.ws_net_profit)] + │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 15 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([wp_web_page_sk@0], 6), input_partitions=2 @@ -8488,7 +8592,7 @@ mod tests { │ CoalescePartitionsExec │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(ss_sold_year@0, ws_sold_year@0), (ss_item_sk@1, ws_item_sk@1), (ss_customer_sk@2, ws_customer_sk@2)], projection=[ss_sold_year@0, ss_item_sk@1, ss_customer_sk@2, ss_qty@3, ss_wc@4, ss_sp@5, ws_qty@9, ws_wc@10, ws_sp@11] │ CoalescePartitionsExec - │ [Stage 5] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 5] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 │ 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 9] => NetworkShuffleExec: output_partitions=3, input_tasks=2 @@ -8496,10 +8600,11 @@ mod tests { │ AggregateExec: mode=FinalPartitioned, gby=[d_year@0 as d_year, cs_item_sk@1 as cs_item_sk, cs_bill_customer_sk@2 as cs_bill_customer_sk], aggr=[sum(catalog_sales.cs_quantity), sum(catalog_sales.cs_wholesale_cost), sum(catalog_sales.cs_sales_price)] │ [Stage 14] => NetworkShuffleExec: output_partitions=3, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ ProjectionExec: expr=[d_year@0 as ss_sold_year, ss_item_sk@1 as ss_item_sk, ss_customer_sk@2 as ss_customer_sk, sum(store_sales.ss_quantity)@3 as ss_qty, sum(store_sales.ss_wholesale_cost)@4 as ss_wc, sum(store_sales.ss_sales_price)@5 as ss_sp] - │ AggregateExec: mode=FinalPartitioned, gby=[d_year@0 as d_year, ss_item_sk@1 as ss_item_sk, ss_customer_sk@2 as ss_customer_sk], aggr=[sum(store_sales.ss_quantity), sum(store_sales.ss_wholesale_cost), sum(store_sales.ss_sales_price)], ordering_mode=PartiallySorted([0]) - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 5 ── Tasks: t0:[p0] t1:[p1] + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_year@0 as ss_sold_year, ss_item_sk@1 as ss_item_sk, ss_customer_sk@2 as ss_customer_sk, sum(store_sales.ss_quantity)@3 as ss_qty, sum(store_sales.ss_wholesale_cost)@4 as ss_wc, sum(store_sales.ss_sales_price)@5 as ss_sp] + │ AggregateExec: mode=FinalPartitioned, gby=[d_year@0 as d_year, ss_item_sk@1 as ss_item_sk, ss_customer_sk@2 as ss_customer_sk], aggr=[sum(store_sales.ss_quantity), sum(store_sales.ss_wholesale_cost), sum(store_sales.ss_sales_price)], ordering_mode=PartiallySorted([0]) + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([d_year@0, ss_item_sk@1, ss_customer_sk@2], 6), input_partitions=3 @@ -8604,18 +8709,19 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c_last_name@0 ASC, c_first_name@1 ASC, substr(ms.s_city,Int64(1),Int64(30))@2 ASC, profit@5 ASC, ss_ticket_number@3 ASC NULLS LAST], fetch=100 - │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] - │ SortExec: TopK(fetch=100), expr=[c_last_name@0 ASC, c_first_name@1 ASC, substr(ms.s_city,Int64(1),Int64(30))@2 ASC, profit@5 ASC, ss_ticket_number@3 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[c_last_name@5 as c_last_name, c_first_name@4 as c_first_name, substr(s_city@1, 1, 30) as substr(ms.s_city,Int64(1),Int64(30)), ss_ticket_number@0 as ss_ticket_number, amt@2 as amt, profit@3 as profit] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@3)], projection=[ss_ticket_number@0, s_city@2, amt@3, profit@4, c_first_name@6, c_last_name@7] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, 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_first_name, c_last_name], file_type=parquet + ┌───── Stage 6 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [c_last_name@0 ASC, c_first_name@1 ASC, substr(ms.s_city,Int64(1),Int64(30))@2 ASC, profit@5 ASC, ss_ticket_number@3 ASC NULLS LAST] + │ SortExec: TopK(fetch=100), expr=[c_last_name@0 ASC, c_first_name@1 ASC, substr(ms.s_city,Int64(1),Int64(30))@2 ASC, profit@5 ASC, ss_ticket_number@3 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[c_last_name@5 as c_last_name, c_first_name@4 as c_first_name, substr(s_city@1, 1, 30) as substr(ms.s_city,Int64(1),Int64(30)), ss_ticket_number@0 as ss_ticket_number, amt@2 as amt, profit@3 as profit] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@3)], projection=[ss_ticket_number@0, s_city@2, amt@3, profit@4, c_first_name@6, c_last_name@7] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, 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_first_name, c_last_name], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 @@ -8882,17 +8988,18 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [c_customer_id@0 ASC NULLS LAST, c_salutation@1 ASC NULLS LAST, c_first_name@2 ASC NULLS LAST, c_last_name@3 ASC NULLS LAST, ca_street_number@4 ASC NULLS LAST, ca_street_name@5 ASC NULLS LAST, ca_street_type@6 ASC NULLS LAST, ca_suite_number@7 ASC NULLS LAST, ca_city@8 ASC NULLS LAST, ca_county@9 ASC NULLS LAST, ca_state@10 ASC NULLS LAST, ca_zip@11 ASC NULLS LAST, ca_country@12 ASC NULLS LAST, ca_gmt_offset@13 ASC NULLS LAST, ca_location_type@14 ASC NULLS LAST, ctr_total_return@15 ASC NULLS LAST], fetch=100 - │ [Stage 11] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 11] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[c_customer_id@0 ASC NULLS LAST, c_salutation@1 ASC NULLS LAST, c_first_name@2 ASC NULLS LAST, c_last_name@3 ASC NULLS LAST, ca_street_number@4 ASC NULLS LAST, ca_street_name@5 ASC NULLS LAST, ca_street_type@6 ASC NULLS LAST, ca_suite_number@7 ASC NULLS LAST, ca_city@8 ASC NULLS LAST, ca_county@9 ASC NULLS LAST, ca_zip@11 ASC NULLS LAST, ca_country@12 ASC NULLS LAST, ca_gmt_offset@13 ASC NULLS LAST, ca_location_type@14 ASC NULLS LAST, ctr_total_return@15 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[c_customer_id@12 as c_customer_id, c_salutation@13 as c_salutation, c_first_name@14 as c_first_name, c_last_name@15 as c_last_name, ca_street_number@1 as ca_street_number, ca_street_name@2 as ca_street_name, ca_street_type@3 as ca_street_type, ca_suite_number@4 as ca_suite_number, ca_city@5 as ca_city, ca_county@6 as ca_county, ca_state@7 as ca_state, ca_zip@8 as ca_zip, ca_country@9 as ca_country, ca_gmt_offset@10 as ca_gmt_offset, ca_location_type@11 as ca_location_type, ctr_total_return@0 as ctr_total_return] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ctr_state@0, ctr_state@1)], filter=CAST(ctr_total_return@0 AS Decimal128(30, 15)) > avg(ctr2.ctr_total_return) * Float64(1.2)@1, projection=[ctr_total_return@1, ca_street_number@2, ca_street_name@3, ca_street_type@4, ca_suite_number@5, ca_city@6, ca_county@7, ca_state@8, ca_zip@9, ca_country@10, ca_gmt_offset@11, ca_location_type@12, c_customer_id@13, c_salutation@14, c_first_name@15, c_last_name@16] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[CAST(CAST(avg(ctr2.ctr_total_return)@1 AS Float64) * 1.2 AS Decimal128(30, 15)) as avg(ctr2.ctr_total_return) * Float64(1.2), ctr_state@0 as ctr_state] - │ AggregateExec: mode=FinalPartitioned, gby=[ctr_state@0 as ctr_state], aggr=[avg(ctr2.ctr_total_return)] - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + ┌───── Stage 11 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [c_customer_id@0 ASC NULLS LAST, c_salutation@1 ASC NULLS LAST, c_first_name@2 ASC NULLS LAST, c_last_name@3 ASC NULLS LAST, ca_street_number@4 ASC NULLS LAST, ca_street_name@5 ASC NULLS LAST, ca_street_type@6 ASC NULLS LAST, ca_suite_number@7 ASC NULLS LAST, ca_city@8 ASC NULLS LAST, ca_county@9 ASC NULLS LAST, ca_state@10 ASC NULLS LAST, ca_zip@11 ASC NULLS LAST, ca_country@12 ASC NULLS LAST, ca_gmt_offset@13 ASC NULLS LAST, ca_location_type@14 ASC NULLS LAST, ctr_total_return@15 ASC NULLS LAST] + │ SortExec: TopK(fetch=100), expr=[c_customer_id@0 ASC NULLS LAST, c_salutation@1 ASC NULLS LAST, c_first_name@2 ASC NULLS LAST, c_last_name@3 ASC NULLS LAST, ca_street_number@4 ASC NULLS LAST, ca_street_name@5 ASC NULLS LAST, ca_street_type@6 ASC NULLS LAST, ca_suite_number@7 ASC NULLS LAST, ca_city@8 ASC NULLS LAST, ca_county@9 ASC NULLS LAST, ca_zip@11 ASC NULLS LAST, ca_country@12 ASC NULLS LAST, ca_gmt_offset@13 ASC NULLS LAST, ca_location_type@14 ASC NULLS LAST, ctr_total_return@15 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[c_customer_id@12 as c_customer_id, c_salutation@13 as c_salutation, c_first_name@14 as c_first_name, c_last_name@15 as c_last_name, ca_street_number@1 as ca_street_number, ca_street_name@2 as ca_street_name, ca_street_type@3 as ca_street_type, ca_suite_number@4 as ca_suite_number, ca_city@5 as ca_city, ca_county@6 as ca_county, ca_state@7 as ca_state, ca_zip@8 as ca_zip, ca_country@9 as ca_country, ca_gmt_offset@10 as ca_gmt_offset, ca_location_type@11 as ca_location_type, ctr_total_return@0 as ctr_total_return] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ctr_state@0, ctr_state@1)], filter=CAST(ctr_total_return@0 AS Decimal128(30, 15)) > avg(ctr2.ctr_total_return) * Float64(1.2)@1, projection=[ctr_total_return@1, ca_street_number@2, ca_street_name@3, ca_street_type@4, ca_suite_number@5, ca_city@6, ca_county@7, ca_state@8, ca_zip@9, ca_country@10, ca_gmt_offset@11, ca_location_type@12, c_customer_id@13, c_salutation@14, c_first_name@15, c_last_name@16] + │ CoalescePartitionsExec + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ ProjectionExec: expr=[CAST(CAST(avg(ctr2.ctr_total_return)@1 AS Float64) * 1.2 AS Decimal128(30, 15)) as avg(ctr2.ctr_total_return) * Float64(1.2), ctr_state@0 as ctr_state] + │ AggregateExec: mode=FinalPartitioned, gby=[ctr_state@0 as ctr_state], aggr=[avg(ctr2.ctr_total_return)] + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=2 └────────────────────────────────────────────────── ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 @@ -8989,12 +9096,13 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [i_item_id@0 ASC NULLS LAST], fetch=100 - │ [Stage 5] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 5] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC NULLS LAST], preserve_partitioning=[true] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_current_price@2 as i_current_price], aggr=[] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 5 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [i_item_id@0 ASC NULLS LAST] + │ SortExec: TopK(fetch=100), expr=[i_item_id@0 ASC NULLS LAST], preserve_partitioning=[true] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_current_price@2 as i_current_price], aggr=[] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, i_current_price@2], 6), input_partitions=2 @@ -9040,21 +9148,22 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [item_id@0 ASC, sr_item_qty@1 ASC], fetch=100 - │ [Stage 17] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 17] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 17 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[item_id@0 ASC, sr_item_qty@1 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[item_id@1 as item_id, sr_item_qty@2 as sr_item_qty, sr_item_qty@2 / __common_expr_7@0 / 3 * 100 as sr_dev, cr_item_qty@3 as cr_item_qty, cr_item_qty@3 / __common_expr_7@0 / 3 * 100 as cr_dev, wr_item_qty@4 as wr_item_qty, wr_item_qty@4 / __common_expr_7@0 / 3 * 100 as wr_dev, __common_expr_7@0 / 3 as average] - │ ProjectionExec: expr=[sr_item_qty@2 + cr_item_qty@3 + wr_item_qty@0 as __common_expr_7, item_id@1 as item_id, sr_item_qty@2 as sr_item_qty, cr_item_qty@3 as cr_item_qty, wr_item_qty@0 as wr_item_qty] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(item_id@0, item_id@0)], projection=[wr_item_qty@1, item_id@2, sr_item_qty@3, cr_item_qty@4] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=1 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(item_id@0, item_id@0)], projection=[item_id@0, sr_item_qty@1, cr_item_qty@3] + ┌───── Stage 17 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [item_id@0 ASC, sr_item_qty@1 ASC] + │ SortExec: TopK(fetch=100), expr=[item_id@0 ASC, sr_item_qty@1 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[item_id@1 as item_id, sr_item_qty@2 as sr_item_qty, sr_item_qty@2 / __common_expr_7@0 / 3 * 100 as sr_dev, cr_item_qty@3 as cr_item_qty, cr_item_qty@3 / __common_expr_7@0 / 3 * 100 as cr_dev, wr_item_qty@4 as wr_item_qty, wr_item_qty@4 / __common_expr_7@0 / 3 * 100 as wr_dev, __common_expr_7@0 / 3 as average] + │ ProjectionExec: expr=[sr_item_qty@2 + cr_item_qty@3 + wr_item_qty@0 as __common_expr_7, item_id@1 as item_id, sr_item_qty@2 as sr_item_qty, cr_item_qty@3 as cr_item_qty, wr_item_qty@0 as wr_item_qty] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(item_id@0, item_id@0)], projection=[wr_item_qty@1, item_id@2, sr_item_qty@3, cr_item_qty@4] │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ 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)] - │ [Stage 16] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(item_id@0, item_id@0)], projection=[item_id@0, sr_item_qty@1, cr_item_qty@3] + │ CoalescePartitionsExec + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ 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)] + │ [Stage 16] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p5] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 @@ -9064,7 +9173,7 @@ mod tests { │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(web_returns.wr_return_quantity)] │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(d_date@2, d_date@0)], projection=[wr_return_quantity@0, i_item_id@1] │ CoalescePartitionsExec - │ [Stage 3] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 3] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d_week_seq@0, d_week_seq@1)], projection=[d_date@0] │ CoalescePartitionsExec │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 @@ -9074,14 +9183,15 @@ 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 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(wr_returned_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@2)], projection=[wr_return_quantity@1, i_item_id@2, d_date@4] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ 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 3 ── Tasks: t0:[p0] t1:[p1] + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(wr_returned_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@2)], projection=[wr_return_quantity@1, i_item_id@2, d_date@4] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ 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 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 @@ -9205,16 +9315,17 @@ mod tests { ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[customer_id@0 as customer_id, customername@1 as customername] │ SortPreservingMergeExec: [c_customer_id@2 ASC], fetch=100 - │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ SortExec: TopK(fetch=100), expr=[customer_id@0 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[c_customer_id@0 as customer_id, concat(concat(CASE WHEN c_last_name@2 IS NOT NULL THEN c_last_name@2 ELSE END, , ), CASE WHEN c_first_name@1 IS NOT NULL THEN c_first_name@1 ELSE END) as customername, c_customer_id@0 as c_customer_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@4, sr_cdemo_sk@0)], projection=[c_customer_id@0, c_first_name@1, c_last_name@2] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ 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_cdemo_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 6 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ SortPreservingMergeExec: [c_customer_id@2 ASC] + │ SortExec: TopK(fetch=100), expr=[customer_id@0 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[c_customer_id@0 as customer_id, concat(concat(CASE WHEN c_last_name@2 IS NOT NULL THEN c_last_name@2 ELSE END, , ), CASE WHEN c_first_name@1 IS NOT NULL THEN c_first_name@1 ELSE END) as customername, c_customer_id@0 as c_customer_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@4, sr_cdemo_sk@0)], projection=[c_customer_id@0, c_first_name@1, c_last_name@2] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ 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_cdemo_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 @@ -9269,13 +9380,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [substr(reason.r_reason_desc,Int64(1),Int64(20))@0 ASC NULLS LAST, avg1@1 ASC NULLS LAST, avg2@2 ASC NULLS LAST, avg(web_returns.wr_fee)@3 ASC NULLS LAST], fetch=100 - │ [Stage 10] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 10] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[substr(reason.r_reason_desc,Int64(1),Int64(20))@0 ASC NULLS LAST, avg1@1 ASC NULLS LAST, avg2@2 ASC NULLS LAST, avg(web_returns.wr_fee)@3 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[substr(r_reason_desc@0, 1, 20) as substr(reason.r_reason_desc,Int64(1),Int64(20)), avg(web_sales.ws_quantity)@1 as avg1, avg(web_returns.wr_refunded_cash)@2 as avg2, avg(web_returns.wr_fee)@3 as avg(web_returns.wr_fee)] - │ AggregateExec: mode=FinalPartitioned, gby=[r_reason_desc@0 as r_reason_desc], aggr=[avg(web_sales.ws_quantity), avg(web_returns.wr_refunded_cash), avg(web_returns.wr_fee)] - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 10 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [substr(reason.r_reason_desc,Int64(1),Int64(20))@0 ASC NULLS LAST, avg1@1 ASC NULLS LAST, avg2@2 ASC NULLS LAST, avg(web_returns.wr_fee)@3 ASC NULLS LAST] + │ SortExec: TopK(fetch=100), expr=[substr(reason.r_reason_desc,Int64(1),Int64(20))@0 ASC NULLS LAST, avg1@1 ASC NULLS LAST, avg2@2 ASC NULLS LAST, avg(web_returns.wr_fee)@3 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[substr(r_reason_desc@0, 1, 20) as substr(reason.r_reason_desc,Int64(1),Int64(20)), avg(web_sales.ws_quantity)@1 as avg1, avg(web_returns.wr_refunded_cash)@2 as avg2, avg(web_returns.wr_fee)@3 as avg(web_returns.wr_fee)] + │ AggregateExec: mode=FinalPartitioned, gby=[r_reason_desc@0 as r_reason_desc], aggr=[avg(web_sales.ws_quantity), avg(web_returns.wr_refunded_cash), avg(web_returns.wr_fee)] + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([r_reason_desc@0], 6), input_partitions=3 @@ -9371,21 +9483,22 @@ mod tests { │ ProjectionExec: expr=[count(Int64(1))@0 as count(*)] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 12] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 12] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] - │ ProjectionExec: expr=[] - │ HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(c_last_name@0, c_last_name@0), (c_first_name@1, c_first_name@1), (d_date@2, d_date@2)], NullsEqual: true - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ AggregateExec: mode=SinglePartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] - │ HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(c_last_name@0, c_last_name@0), (c_first_name@1, c_first_name@1), (d_date@2, d_date@2)], NullsEqual: true - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ AggregateExec: mode=SinglePartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] - │ AggregateExec: mode=FinalPartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] - │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 12 ── Tasks: t0:[p0] t1:[p1] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] + │ ProjectionExec: expr=[] + │ HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(c_last_name@0, c_last_name@0), (c_first_name@1, c_first_name@1), (d_date@2, d_date@2)], NullsEqual: true + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ AggregateExec: mode=SinglePartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] + │ HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(c_last_name@0, c_last_name@0), (c_first_name@1, c_first_name@1), (d_date@2, d_date@2)], NullsEqual: true + │ CoalescePartitionsExec + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ AggregateExec: mode=SinglePartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] + │ AggregateExec: mode=FinalPartitioned, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] + │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 @@ -9497,61 +9610,62 @@ mod tests { │ ProjectionExec: expr=[count(Int64(1))@0 as h12_to_12_30] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[h8_30_to_9@1 as h8_30_to_9, h9_to_9_30@2 as h9_to_9_30, h9_30_to_10@3 as h9_30_to_10, h10_to_10_30@4 as h10_to_10_30, h10_30_to_11@5 as h10_30_to_11, h11_to_11_30@6 as h11_to_11_30, h11_30_to_12@0 as h11_30_to_12] │ CrossJoinExec │ ProjectionExec: expr=[count(Int64(1))@0 as h11_30_to_12] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 8] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 8] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[h8_30_to_9@1 as h8_30_to_9, h9_to_9_30@2 as h9_to_9_30, h9_30_to_10@3 as h9_30_to_10, h10_to_10_30@4 as h10_to_10_30, h10_30_to_11@5 as h10_30_to_11, h11_to_11_30@0 as h11_to_11_30] │ CrossJoinExec │ ProjectionExec: expr=[count(Int64(1))@0 as h11_to_11_30] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 12] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 12] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[h8_30_to_9@1 as h8_30_to_9, h9_to_9_30@2 as h9_to_9_30, h9_30_to_10@3 as h9_30_to_10, h10_to_10_30@4 as h10_to_10_30, h10_30_to_11@0 as h10_30_to_11] │ CrossJoinExec │ ProjectionExec: expr=[count(Int64(1))@0 as h10_30_to_11] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 16] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 16] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[h8_30_to_9@1 as h8_30_to_9, h9_to_9_30@2 as h9_to_9_30, h9_30_to_10@3 as h9_30_to_10, h10_to_10_30@0 as h10_to_10_30] │ CrossJoinExec │ ProjectionExec: expr=[count(Int64(1))@0 as h10_to_10_30] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 20] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 20] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[h8_30_to_9@1 as h8_30_to_9, h9_to_9_30@2 as h9_to_9_30, h9_30_to_10@0 as h9_30_to_10] │ CrossJoinExec │ ProjectionExec: expr=[count(Int64(1))@0 as h9_30_to_10] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 24] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 24] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ CrossJoinExec │ ProjectionExec: expr=[count(Int64(1))@0 as h8_30_to_9] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 28] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 28] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[count(Int64(1))@0 as h9_to_9_30] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 32] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 32] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] - │ ProjectionExec: expr=[] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ss_sold_time_sk@0)], projection=[ss_store_sk@3] + ┌───── Stage 4 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] + │ ProjectionExec: expr=[] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_sold_time_sk@2, ss_store_sk@4] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ss_sold_time_sk@0)], projection=[ss_store_sk@3] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_time_sk, ss_hdemo_sk, ss_store_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_sold_time_sk@2, ss_store_sk@4] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_time_sk, ss_hdemo_sk, ss_store_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -9577,20 +9691,21 @@ 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_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 4 AND hd_vehicle_count@4 <= 6 OR hd_dep_count@3 = 2 AND hd_vehicle_count@4 <= 4 OR hd_dep_count@3 = 0 AND hd_vehicle_count@4 <= 2, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 4 AND 4 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 6 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 2 AND 2 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 4 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 0 AND 0 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 2, required_guarantees=[hd_dep_count in (0, 2, 4)] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] - │ ProjectionExec: expr=[] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ss_sold_time_sk@0)], projection=[ss_store_sk@3] + ┌───── Stage 8 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] + │ ProjectionExec: expr=[] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)] │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_sold_time_sk@2, ss_store_sk@4] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ss_sold_time_sk@0)], projection=[ss_store_sk@3] │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_time_sk, ss_hdemo_sk, ss_store_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_sold_time_sk@2, ss_store_sk@4] + │ CoalescePartitionsExec + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_time_sk, ss_hdemo_sk, ss_store_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -9616,20 +9731,21 @@ 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_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 4 AND hd_vehicle_count@4 <= 6 OR hd_dep_count@3 = 2 AND hd_vehicle_count@4 <= 4 OR hd_dep_count@3 = 0 AND hd_vehicle_count@4 <= 2, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 4 AND 4 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 6 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 2 AND 2 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 4 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 0 AND 0 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 2, required_guarantees=[hd_dep_count in (0, 2, 4)] └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] - │ ProjectionExec: expr=[] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)] - │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ss_sold_time_sk@0)], projection=[ss_store_sk@3] + ┌───── Stage 12 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] + │ ProjectionExec: expr=[] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)] │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_sold_time_sk@2, ss_store_sk@4] + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ss_sold_time_sk@0)], projection=[ss_store_sk@3] │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_time_sk, ss_hdemo_sk, ss_store_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_sold_time_sk@2, ss_store_sk@4] + │ CoalescePartitionsExec + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_time_sk, ss_hdemo_sk, ss_store_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -9655,20 +9771,21 @@ 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_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 4 AND hd_vehicle_count@4 <= 6 OR hd_dep_count@3 = 2 AND hd_vehicle_count@4 <= 4 OR hd_dep_count@3 = 0 AND hd_vehicle_count@4 <= 2, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 4 AND 4 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 6 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 2 AND 2 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 4 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 0 AND 0 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 2, required_guarantees=[hd_dep_count in (0, 2, 4)] └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] - │ ProjectionExec: expr=[] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)] - │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ss_sold_time_sk@0)], projection=[ss_store_sk@3] + ┌───── Stage 16 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] + │ ProjectionExec: expr=[] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)] │ CoalescePartitionsExec - │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_sold_time_sk@2, ss_store_sk@4] + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ss_sold_time_sk@0)], projection=[ss_store_sk@3] │ CoalescePartitionsExec - │ [Stage 15] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_time_sk, ss_hdemo_sk, ss_store_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_sold_time_sk@2, ss_store_sk@4] + │ CoalescePartitionsExec + │ [Stage 15] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_time_sk, ss_hdemo_sk, ss_store_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 13 ── Tasks: t0:[p0..p8] t1:[p9..p17] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -9694,20 +9811,21 @@ 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_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 4 AND hd_vehicle_count@4 <= 6 OR hd_dep_count@3 = 2 AND hd_vehicle_count@4 <= 4 OR hd_dep_count@3 = 0 AND hd_vehicle_count@4 <= 2, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 4 AND 4 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 6 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 2 AND 2 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 4 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 0 AND 0 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 2, required_guarantees=[hd_dep_count in (0, 2, 4)] └────────────────────────────────────────────────── - ┌───── Stage 20 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] - │ ProjectionExec: expr=[] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)] - │ CoalescePartitionsExec - │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ss_sold_time_sk@0)], projection=[ss_store_sk@3] + ┌───── Stage 20 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] + │ ProjectionExec: expr=[] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)] │ CoalescePartitionsExec - │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_sold_time_sk@2, ss_store_sk@4] + │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ss_sold_time_sk@0)], projection=[ss_store_sk@3] │ CoalescePartitionsExec - │ [Stage 19] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_time_sk, ss_hdemo_sk, ss_store_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_sold_time_sk@2, ss_store_sk@4] + │ CoalescePartitionsExec + │ [Stage 19] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_time_sk, ss_hdemo_sk, ss_store_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 17 ── Tasks: t0:[p0..p8] t1:[p9..p17] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -9733,20 +9851,21 @@ 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_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 4 AND hd_vehicle_count@4 <= 6 OR hd_dep_count@3 = 2 AND hd_vehicle_count@4 <= 4 OR hd_dep_count@3 = 0 AND hd_vehicle_count@4 <= 2, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 4 AND 4 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 6 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 2 AND 2 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 4 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 0 AND 0 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 2, required_guarantees=[hd_dep_count in (0, 2, 4)] └────────────────────────────────────────────────── - ┌───── Stage 24 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] - │ ProjectionExec: expr=[] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)] - │ CoalescePartitionsExec - │ [Stage 21] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ss_sold_time_sk@0)], projection=[ss_store_sk@3] + ┌───── Stage 24 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] + │ ProjectionExec: expr=[] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)] │ CoalescePartitionsExec - │ [Stage 22] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_sold_time_sk@2, ss_store_sk@4] + │ [Stage 21] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ss_sold_time_sk@0)], projection=[ss_store_sk@3] │ CoalescePartitionsExec - │ [Stage 23] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_time_sk, ss_hdemo_sk, ss_store_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 22] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_sold_time_sk@2, ss_store_sk@4] + │ CoalescePartitionsExec + │ [Stage 23] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_time_sk, ss_hdemo_sk, ss_store_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 21 ── Tasks: t0:[p0..p8] t1:[p9..p17] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -9772,20 +9891,21 @@ 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_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 4 AND hd_vehicle_count@4 <= 6 OR hd_dep_count@3 = 2 AND hd_vehicle_count@4 <= 4 OR hd_dep_count@3 = 0 AND hd_vehicle_count@4 <= 2, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 4 AND 4 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 6 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 2 AND 2 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 4 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 0 AND 0 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 2, required_guarantees=[hd_dep_count in (0, 2, 4)] └────────────────────────────────────────────────── - ┌───── Stage 28 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] - │ ProjectionExec: expr=[] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)] - │ CoalescePartitionsExec - │ [Stage 25] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ss_sold_time_sk@0)], projection=[ss_store_sk@3] + ┌───── Stage 28 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] + │ ProjectionExec: expr=[] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)] │ CoalescePartitionsExec - │ [Stage 26] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_sold_time_sk@2, ss_store_sk@4] + │ [Stage 25] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ss_sold_time_sk@0)], projection=[ss_store_sk@3] │ CoalescePartitionsExec - │ [Stage 27] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_time_sk, ss_hdemo_sk, ss_store_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 26] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_sold_time_sk@2, ss_store_sk@4] + │ CoalescePartitionsExec + │ [Stage 27] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_time_sk, ss_hdemo_sk, ss_store_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 25 ── Tasks: t0:[p0..p8] t1:[p9..p17] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -9811,20 +9931,21 @@ 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_dep_count, hd_vehicle_count], file_type=parquet, predicate=hd_dep_count@3 = 4 AND hd_vehicle_count@4 <= 6 OR hd_dep_count@3 = 2 AND hd_vehicle_count@4 <= 4 OR hd_dep_count@3 = 0 AND hd_vehicle_count@4 <= 2, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 4 AND 4 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 6 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 2 AND 2 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 4 OR hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 0 AND 0 <= hd_dep_count_max@1 AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_min@4 <= 2, required_guarantees=[hd_dep_count in (0, 2, 4)] └────────────────────────────────────────────────── - ┌───── Stage 32 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] - │ ProjectionExec: expr=[] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)] - │ CoalescePartitionsExec - │ [Stage 29] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ss_sold_time_sk@0)], projection=[ss_store_sk@3] + ┌───── Stage 32 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] + │ ProjectionExec: expr=[] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)] │ CoalescePartitionsExec - │ [Stage 30] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_sold_time_sk@2, ss_store_sk@4] + │ [Stage 29] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ss_sold_time_sk@0)], projection=[ss_store_sk@3] │ CoalescePartitionsExec - │ [Stage 31] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_time_sk, ss_hdemo_sk, ss_store_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 30] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_sold_time_sk@2, ss_store_sk@4] + │ CoalescePartitionsExec + │ [Stage 31] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_time_sk, ss_hdemo_sk, ss_store_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 29 ── Tasks: t0:[p0..p8] t1:[p9..p17] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -9922,26 +10043,27 @@ mod tests { │ ProjectionExec: expr=[count(Int64(1))@0 as amc] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ ProjectionExec: expr=[count(Int64(1))@0 as pmc] │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 8] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 8] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] - │ ProjectionExec: expr=[] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_page.wp_web_page_sk AS Float64)@1, ws_web_page_sk@0)] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ws_sold_time_sk@0)], projection=[ws_web_page_sk@3] + ┌───── Stage 4 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] + │ ProjectionExec: expr=[] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_page.wp_web_page_sk AS Float64)@1, ws_web_page_sk@0)] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ws_ship_hdemo_sk@1)], projection=[ws_sold_time_sk@2, ws_web_page_sk@4] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ws_sold_time_sk@0)], projection=[ws_web_page_sk@3] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_time_sk, ws_ship_hdemo_sk, ws_web_page_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ws_ship_hdemo_sk@1)], projection=[ws_sold_time_sk@2, ws_web_page_sk@4] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_time_sk, ws_ship_hdemo_sk, ws_web_page_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -9967,20 +10089,21 @@ 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_dep_count], file_type=parquet, predicate=hd_dep_count@3 = 6, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 6 AND 6 <= hd_dep_count_max@1, required_guarantees=[hd_dep_count in (6)] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] - │ ProjectionExec: expr=[] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_page.wp_web_page_sk AS Float64)@1, ws_web_page_sk@0)] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ws_sold_time_sk@0)], projection=[ws_web_page_sk@3] + ┌───── Stage 8 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] + │ ProjectionExec: expr=[] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_page.wp_web_page_sk AS Float64)@1, ws_web_page_sk@0)] │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ws_ship_hdemo_sk@1)], projection=[ws_sold_time_sk@2, ws_web_page_sk@4] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ws_sold_time_sk@0)], projection=[ws_web_page_sk@3] │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_time_sk, ws_ship_hdemo_sk, ws_web_page_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ws_ship_hdemo_sk@1)], projection=[ws_sold_time_sk@2, ws_web_page_sk@4] + │ CoalescePartitionsExec + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_time_sk, ws_ship_hdemo_sk, ws_web_page_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -10015,13 +10138,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [returns_loss@3 DESC] - │ [Stage 8] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 8] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: expr=[returns_loss@3 DESC], preserve_partitioning=[true] - │ ProjectionExec: expr=[cc_call_center_id@0 as call_center, cc_name@1 as call_center_name, cc_manager@2 as manager, sum(catalog_returns.cr_net_loss)@5 as returns_loss] - │ AggregateExec: mode=FinalPartitioned, gby=[cc_call_center_id@0 as cc_call_center_id, cc_name@1 as cc_name, cc_manager@2 as cc_manager, cd_marital_status@3 as cd_marital_status, cd_education_status@4 as cd_education_status], aggr=[sum(catalog_returns.cr_net_loss)] - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 8 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [returns_loss@3 DESC] + │ SortExec: expr=[returns_loss@3 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[cc_call_center_id@0 as call_center, cc_name@1 as call_center_name, cc_manager@2 as manager, sum(catalog_returns.cr_net_loss)@5 as returns_loss] + │ AggregateExec: mode=FinalPartitioned, gby=[cc_call_center_id@0 as cc_call_center_id, cc_name@1 as cc_name, cc_manager@2 as cc_manager, cd_marital_status@3 as cd_marital_status, cd_education_status@4 as cd_education_status], aggr=[sum(catalog_returns.cr_net_loss)] + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([cc_call_center_id@0, cc_name@1, cc_manager@2, cd_marital_status@3, cd_education_status@4], 6), input_partitions=3 @@ -10100,22 +10224,23 @@ mod tests { │ GlobalLimitExec: skip=0, fetch=100 │ AggregateExec: mode=Final, gby=[], aggr=[sum(web_sales.ws_ext_discount_amt)] │ CoalescePartitionsExec - │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[sum(web_sales.ws_ext_discount_amt)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_item_sk@1, i_item_sk@1)], filter=CAST(ws_ext_discount_amt@0 AS Decimal128(30, 15)) > Float64(1.3) * avg(web_sales.ws_ext_discount_amt)@1, projection=[ws_ext_discount_amt@2] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_ext_discount_amt@3, i_item_sk@4] + ┌───── Stage 6 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(web_sales.ws_ext_discount_amt)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_item_sk@1, i_item_sk@1)], filter=CAST(ws_ext_discount_amt@0 AS Decimal128(30, 15)) > Float64(1.3) * avg(web_sales.ws_ext_discount_amt)@1, projection=[ws_ext_discount_amt@2] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ws_sold_date_sk@1 as ws_sold_date_sk, ws_ext_discount_amt@2 as ws_ext_discount_amt, i_item_sk@0 as i_item_sk] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_item_sk@0, ws_sold_date_sk@1, ws_ext_discount_amt@3] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_discount_amt], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_ext_discount_amt@3, i_item_sk@4] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ ProjectionExec: expr=[ws_sold_date_sk@1 as ws_sold_date_sk, ws_ext_discount_amt@2 as ws_ext_discount_amt, i_item_sk@0 as i_item_sk] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_item_sk@0, ws_sold_date_sk@1, ws_ext_discount_amt@3] + │ CoalescePartitionsExec + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_discount_amt], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -10164,13 +10289,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [sumsales@1 ASC, ss_customer_sk@0 ASC], fetch=100 - │ [Stage 5] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 5] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[sumsales@1 ASC, ss_customer_sk@0 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[ss_customer_sk@0 as ss_customer_sk, sum(t.act_sales)@1 as sumsales] - │ AggregateExec: mode=FinalPartitioned, gby=[ss_customer_sk@0 as ss_customer_sk], aggr=[sum(t.act_sales)] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 5 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [sumsales@1 ASC, ss_customer_sk@0 ASC] + │ SortExec: TopK(fetch=100), expr=[sumsales@1 ASC, ss_customer_sk@0 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[ss_customer_sk@0 as ss_customer_sk, sum(t.act_sales)@1 as sumsales] + │ AggregateExec: mode=FinalPartitioned, gby=[ss_customer_sk@0 as ss_customer_sk], aggr=[sum(t.act_sales)] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([ss_customer_sk@0], 6), input_partitions=3 @@ -10222,23 +10348,24 @@ mod tests { │ CoalescePartitionsExec │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(ws_order_number@1, ws_order_number@1)], filter=ws_warehouse_sk@0 != ws_warehouse_sk@1, projection=[ws_order_number@1, ws_ext_ship_cost@2, ws_net_profit@3] │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ 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_warehouse_sk, ws_order_number], file_type=parquet │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 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 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_site.web_site_sk AS Float64)@1, ws_web_site_sk@0)], projection=[ws_warehouse_sk@3, ws_order_number@4, ws_ext_ship_cost@5, ws_net_profit@6] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer_address.ca_address_sk AS Float64)@1, ws_ship_addr_sk@0)], projection=[ws_web_site_sk@3, ws_warehouse_sk@4, ws_order_number@5, ws_ext_ship_cost@6, ws_net_profit@7] + ┌───── Stage 4 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_site.web_site_sk AS Float64)@1, ws_web_site_sk@0)], projection=[ws_warehouse_sk@3, ws_order_number@4, ws_ext_ship_cost@5, ws_net_profit@6] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_ship_date_sk@0)], projection=[ws_ship_addr_sk@3, ws_web_site_sk@4, ws_warehouse_sk@5, ws_order_number@6, ws_ext_ship_cost@7, ws_net_profit@8] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer_address.ca_address_sk AS Float64)@1, ws_ship_addr_sk@0)], projection=[ws_web_site_sk@3, ws_warehouse_sk@4, ws_order_number@5, ws_ext_ship_cost@6, ws_net_profit@7] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_ship_date_sk, ws_ship_addr_sk, ws_web_site_sk, ws_warehouse_sk, ws_order_number, ws_ext_ship_cost, ws_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_ship_date_sk@0)], projection=[ws_ship_addr_sk@3, ws_web_site_sk@4, ws_warehouse_sk@5, ws_order_number@6, ws_ext_ship_cost@7, ws_net_profit@8] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_ship_date_sk, ws_ship_addr_sk, ws_web_site_sk, ws_warehouse_sk, ws_order_number, ws_ext_ship_cost, ws_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -10284,7 +10411,7 @@ mod tests { │ CoalescePartitionsExec │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(ws_order_number@0, ws_order_number@0)] │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 │ 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 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 @@ -10296,18 +10423,19 @@ mod tests { │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_site.web_site_sk AS Float64)@1, ws_web_site_sk@0)], projection=[ws_order_number@3, ws_ext_ship_cost@4, ws_net_profit@5] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer_address.ca_address_sk AS Float64)@1, ws_ship_addr_sk@0)], projection=[ws_web_site_sk@3, ws_order_number@4, ws_ext_ship_cost@5, ws_net_profit@6] + ┌───── Stage 4 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_site.web_site_sk AS Float64)@1, ws_web_site_sk@0)], projection=[ws_order_number@3, ws_ext_ship_cost@4, ws_net_profit@5] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_ship_date_sk@0)], projection=[ws_ship_addr_sk@3, ws_web_site_sk@4, ws_order_number@5, ws_ext_ship_cost@6, ws_net_profit@7] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer_address.ca_address_sk AS Float64)@1, ws_ship_addr_sk@0)], projection=[ws_web_site_sk@3, ws_order_number@4, ws_ext_ship_cost@5, ws_net_profit@6] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_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 ] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_ship_date_sk@0)], projection=[ws_ship_addr_sk@3, ws_web_site_sk@4, ws_order_number@5, ws_ext_ship_cost@6, ws_net_profit@7] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_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 ] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -10365,22 +10493,23 @@ mod tests { │ GlobalLimitExec: skip=0, fetch=100 │ AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1))] │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] - │ ProjectionExec: expr=[] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ss_sold_time_sk@0)], projection=[ss_store_sk@3] + ┌───── Stage 4 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] + │ ProjectionExec: expr=[] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_sold_time_sk@2, ss_store_sk@4] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@1, ss_sold_time_sk@0)], projection=[ss_store_sk@3] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_time_sk, ss_hdemo_sk, ss_store_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_sold_time_sk@2, ss_store_sk@4] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=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_time_sk, ss_hdemo_sk, ss_store_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 @@ -10422,15 +10551,16 @@ mod tests { │ ProjectionExec: expr=[customer_sk@1 IS NOT NULL as __common_expr_1, customer_sk@1 as customer_sk, customer_sk@0 as customer_sk] │ HashJoinExec: mode=CollectLeft, join_type=Full, on=[(customer_sk@0, customer_sk@0), (item_sk@1, item_sk@1)], projection=[customer_sk@0, customer_sk@2] │ CoalescePartitionsExec - │ [Stage 3] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 3] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 │ ProjectionExec: expr=[ss_customer_sk@0 as customer_sk, ss_item_sk@1 as item_sk] │ AggregateExec: mode=FinalPartitioned, gby=[ss_customer_sk@0 as ss_customer_sk, ss_item_sk@1 as ss_item_sk], aggr=[] │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ ProjectionExec: expr=[cs_bill_customer_sk@0 as customer_sk, cs_item_sk@1 as item_sk] - │ AggregateExec: mode=FinalPartitioned, gby=[cs_bill_customer_sk@0 as cs_bill_customer_sk, cs_item_sk@1 as cs_item_sk], aggr=[] - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 3 ── Tasks: t0:[p0] t1:[p1] + │ CoalescePartitionsExec + │ ProjectionExec: expr=[cs_bill_customer_sk@0 as customer_sk, cs_item_sk@1 as item_sk] + │ AggregateExec: mode=FinalPartitioned, gby=[cs_bill_customer_sk@0 as cs_bill_customer_sk, cs_item_sk@1 as cs_item_sk], aggr=[] + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@0, cs_item_sk@1], 6), input_partitions=2 @@ -10523,13 +10653,14 @@ mod tests { assert_snapshot!(display, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [w_substr@0 ASC, sm_type@1 ASC, cc_name_lower@2 ASC], fetch=100 - │ [Stage 6] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ SortExec: TopK(fetch=100), expr=[w_substr@0 ASC, sm_type@1 ASC, cc_name_lower@2 ASC], preserve_partitioning=[true] - │ ProjectionExec: expr=[w_substr@0 as w_substr, sm_type@1 as sm_type, lower(cc_name@2) as cc_name_lower, sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END)@3 as 30 days, sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(30) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END)@4 as 31-60 days, sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(60) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END)@5 as 61-90 days, sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(90) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END)@6 as 91-120 days, sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)@7 as >120 days] - │ AggregateExec: mode=FinalPartitioned, gby=[w_substr@0 as w_substr, sm_type@1 as sm_type, cc_name@2 as cc_name], aggr=[sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(30) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(60) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(90) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 6 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [w_substr@0 ASC, sm_type@1 ASC, cc_name_lower@2 ASC] + │ SortExec: TopK(fetch=100), expr=[w_substr@0 ASC, sm_type@1 ASC, cc_name_lower@2 ASC], preserve_partitioning=[true] + │ ProjectionExec: expr=[w_substr@0 as w_substr, sm_type@1 as sm_type, lower(cc_name@2) as cc_name_lower, sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END)@3 as 30 days, sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(30) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END)@4 as 31-60 days, sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(60) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END)@5 as 61-90 days, sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(90) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END)@6 as 91-120 days, sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)@7 as >120 days] + │ AggregateExec: mode=FinalPartitioned, gby=[w_substr@0 as w_substr, sm_type@1 as sm_type, cc_name@2 as cc_name], aggr=[sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(30) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(30) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(60) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(60) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(90) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(90) AND catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk <= Int64(120) THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN catalog_sales.cs_ship_date_sk - catalog_sales.cs_sold_date_sk > Int64(120) THEN Int64(1) ELSE Int64(0) END)] + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([w_substr@0, sm_type@1, cc_name@2], 6), input_partitions=3 diff --git a/tests/tpch_plans_test.rs b/tests/tpch_plans_test.rs index 19613afe..f9f1c86c 100644 --- a/tests/tpch_plans_test.rs +++ b/tests/tpch_plans_test.rs @@ -20,13 +20,14 @@ mod tests { assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST] - │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] - │ SortExec: expr=[l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus, sum(lineitem.l_quantity)@2 as sum_qty, sum(lineitem.l_extendedprice)@3 as sum_base_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@4 as sum_disc_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax)@5 as sum_charge, avg(lineitem.l_quantity)@6 as avg_qty, avg(lineitem.l_extendedprice)@7 as avg_price, avg(lineitem.l_discount)@8 as avg_disc, count(Int64(1))@9 as count_order] - │ AggregateExec: mode=FinalPartitioned, gby=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST] + │ SortExec: expr=[l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus, sum(lineitem.l_quantity)@2 as sum_qty, sum(lineitem.l_extendedprice)@3 as sum_base_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@4 as sum_disc_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax)@5 as sum_charge, avg(lineitem.l_quantity)@6 as avg_qty, avg(lineitem.l_extendedprice)@7 as avg_price, avg(lineitem.l_discount)@8 as avg_disc, count(Int64(1))@9 as count_order] + │ AggregateExec: mode=FinalPartitioned, gby=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(Int64(1))] + │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] │ RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 12), input_partitions=4 @@ -46,14 +47,15 @@ mod tests { assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST] - │ [Stage 11] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 + │ [Stage 11] => NetworkCoalesceExec: output_partitions=4, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ SortExec: expr=[s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[s_acctbal@5 as s_acctbal, s_name@2 as s_name, n_name@7 as n_name, p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_address@3 as s_address, s_phone@4 as s_phone, s_comment@6 as s_comment] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@1), (ps_supplycost@7, min(partsupp.ps_supplycost)@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, n_name@8] - │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=4 - │ [Stage 10] => NetworkShuffleExec: output_partitions=6, input_tasks=3 + ┌───── Stage 11 ── Tasks: t0:[p0] t1:[p1] t2:[p2] t3:[p3] + │ SortPreservingMergeExec: [s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST] + │ SortExec: expr=[s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[s_acctbal@5 as s_acctbal, s_name@2 as s_name, n_name@7 as n_name, p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_address@3 as s_address, s_phone@4 as s_phone, s_comment@6 as s_comment] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@1), (ps_supplycost@7, min(partsupp.ps_supplycost)@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, n_name@8] + │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + │ [Stage 10] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] │ RepartitionExec: partitioning=Hash([p_partkey@0, ps_supplycost@7], 24), input_partitions=4 @@ -145,13 +147,14 @@ mod tests { assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [revenue@1 DESC, o_orderdate@2 ASC NULLS LAST] - │ [Stage 4] => NetworkCoalesceExec: output_partitions=18, input_tasks=3 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ SortExec: expr=[revenue@1 DESC, o_orderdate@2 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[l_orderkey@0 as l_orderkey, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@3 as revenue, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority] - │ AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + ┌───── Stage 4 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ SortPreservingMergeExec: [revenue@1 DESC, o_orderdate@2 ASC NULLS LAST] + │ SortExec: expr=[revenue@1 DESC, o_orderdate@2 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[l_orderkey@0 as l_orderkey, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@3 as revenue, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority] + │ AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] │ RepartitionExec: partitioning=Hash([l_orderkey@0, o_orderdate@1, o_shippriority@2], 18), input_partitions=4 @@ -195,14 +198,15 @@ mod tests { │ AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))] │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderpriority@1] │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=16, input_tasks=4 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=4, input_tasks=4 │ FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@12 > l_commitdate@11 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] - │ FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet], [/testdata/tpch/plan_sf0.02/orders/10.parquet], [/testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet], [/testdata/tpch/plan_sf0.02/orders/13.parquet], ...]}, projection=[o_orderkey, o_orderdate, o_orderpriority], file_type=parquet, predicate=o_orderdate@4 >= 1993-07-01 AND o_orderdate@4 < 1993-10-01, pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1993-07-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 < 1993-10-01, required_guarantees=[] + ┌───── Stage 1 ── Tasks: t0:[p0] t1:[p1] t2:[p2] t3:[p3] + │ CoalescePartitionsExec + │ FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2] + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] + │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet], [/testdata/tpch/plan_sf0.02/orders/10.parquet], [/testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet], [/testdata/tpch/plan_sf0.02/orders/13.parquet], ...]}, projection=[o_orderkey, o_orderdate, o_orderpriority], file_type=parquet, predicate=o_orderdate@4 >= 1993-07-01 AND o_orderdate@4 < 1993-10-01, pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1993-07-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 < 1993-10-01, required_guarantees=[] └────────────────────────────────────────────────── "); Ok(()) @@ -214,13 +218,14 @@ mod tests { assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [revenue@1 DESC] - │ [Stage 7] => NetworkCoalesceExec: output_partitions=18, input_tasks=3 + │ [Stage 7] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ SortExec: expr=[revenue@1 DESC], preserve_partitioning=[true] - │ ProjectionExec: expr=[n_name@0 as n_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as revenue] - │ AggregateExec: mode=FinalPartitioned, gby=[n_name@0 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + ┌───── Stage 7 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ SortPreservingMergeExec: [revenue@1 DESC] + │ SortExec: expr=[revenue@1 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[n_name@0 as n_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as revenue] + │ AggregateExec: mode=FinalPartitioned, gby=[n_name@0 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 6 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] │ RepartitionExec: partitioning=Hash([n_name@0], 18), input_partitions=4 @@ -284,13 +289,14 @@ mod tests { │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * lineitem.l_discount)@0 as revenue] │ AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=16, input_tasks=4 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=4, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] - │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] - │ FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2, projection=[l_extendedprice@1, l_discount@2] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet], [/testdata/tpch/plan_sf0.02/lineitem/10.parquet], [/testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet], [/testdata/tpch/plan_sf0.02/lineitem/13.parquet], ...]}, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@10 >= 1994-01-01 AND l_shipdate@10 < 1995-01-01 AND l_discount@6 >= Some(5),15,2 AND l_discount@6 <= Some(7),15,2 AND l_quantity@4 < Some(2400),15,2, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1994-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1995-01-01 AND l_discount_null_count@5 != row_count@2 AND l_discount_max@4 >= Some(5),15,2 AND l_discount_null_count@5 != row_count@2 AND l_discount_min@6 <= Some(7),15,2 AND l_quantity_null_count@8 != row_count@2 AND l_quantity_min@7 < Some(2400),15,2, required_guarantees=[] + ┌───── Stage 1 ── Tasks: t0:[p0] t1:[p1] t2:[p2] t3:[p3] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] + │ FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2, projection=[l_extendedprice@1, l_discount@2] + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] + │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet], [/testdata/tpch/plan_sf0.02/lineitem/10.parquet], [/testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet], [/testdata/tpch/plan_sf0.02/lineitem/13.parquet], ...]}, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@10 >= 1994-01-01 AND l_shipdate@10 < 1995-01-01 AND l_discount@6 >= Some(5),15,2 AND l_discount@6 <= Some(7),15,2 AND l_quantity@4 < Some(2400),15,2, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1994-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1995-01-01 AND l_discount_null_count@5 != row_count@2 AND l_discount_max@4 >= Some(5),15,2 AND l_discount_null_count@5 != row_count@2 AND l_discount_min@6 <= Some(7),15,2 AND l_quantity_null_count@8 != row_count@2 AND l_quantity_min@7 < Some(2400),15,2, required_guarantees=[] └────────────────────────────────────────────────── "); Ok(()) @@ -302,13 +308,14 @@ mod tests { assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST] - │ [Stage 7] => NetworkCoalesceExec: output_partitions=18, input_tasks=3 + │ [Stage 7] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ SortExec: expr=[supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year, sum(shipping.volume)@3 as revenue] - │ AggregateExec: mode=FinalPartitioned, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)] - │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + ┌───── Stage 7 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST] + │ SortExec: expr=[supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year, sum(shipping.volume)@3 as revenue] + │ AggregateExec: mode=FinalPartitioned, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)] + │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 6 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] │ RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 18), input_partitions=4 @@ -373,13 +380,14 @@ mod tests { assert_snapshot!(plan, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [o_year@0 ASC NULLS LAST] - │ [Stage 9] => NetworkCoalesceExec: output_partitions=18, input_tasks=3 + │ [Stage 9] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ SortExec: expr=[o_year@0 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[o_year@0 as o_year, sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END)@1 / sum(all_nations.volume)@2 as mkt_share] - │ AggregateExec: mode=FinalPartitioned, gby=[o_year@0 as o_year], aggr=[sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)] - │ [Stage 8] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + ┌───── Stage 9 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ SortPreservingMergeExec: [o_year@0 ASC NULLS LAST] + │ SortExec: expr=[o_year@0 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[o_year@0 as o_year, sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END)@1 / sum(all_nations.volume)@2 as mkt_share] + │ AggregateExec: mode=FinalPartitioned, gby=[o_year@0 as o_year], aggr=[sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)] + │ [Stage 8] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 8 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] │ RepartitionExec: partitioning=Hash([o_year@0], 18), input_partitions=4 @@ -462,13 +470,14 @@ mod tests { assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [nation@0 ASC NULLS LAST, o_year@1 DESC] - │ [Stage 7] => NetworkCoalesceExec: output_partitions=18, input_tasks=3 + │ [Stage 7] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ SortExec: expr=[nation@0 ASC NULLS LAST, o_year@1 DESC], preserve_partitioning=[true] - │ ProjectionExec: expr=[nation@0 as nation, o_year@1 as o_year, sum(profit.amount)@2 as sum_profit] - │ AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] - │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + ┌───── Stage 7 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ SortPreservingMergeExec: [nation@0 ASC NULLS LAST, o_year@1 DESC] + │ SortExec: expr=[nation@0 ASC NULLS LAST, o_year@1 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[nation@0 as nation, o_year@1 as o_year, sum(profit.amount)@2 as sum_profit] + │ AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] + │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 6 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] │ RepartitionExec: partitioning=Hash([nation@0, o_year@1], 18), input_partitions=4 @@ -531,13 +540,14 @@ mod tests { assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [revenue@2 DESC] - │ [Stage 5] => NetworkCoalesceExec: output_partitions=18, input_tasks=3 + │ [Stage 5] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ SortExec: expr=[revenue@2 DESC], preserve_partitioning=[true] - │ ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@7 as revenue, c_acctbal@2 as c_acctbal, n_name@4 as n_name, c_address@5 as c_address, c_phone@3 as c_phone, c_comment@6 as c_comment] - │ AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@2 as c_acctbal, c_phone@3 as c_phone, n_name@4 as n_name, c_address@5 as c_address, c_comment@6 as c_comment], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + ┌───── Stage 5 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ SortPreservingMergeExec: [revenue@2 DESC] + │ SortExec: expr=[revenue@2 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@7 as revenue, c_acctbal@2 as c_acctbal, n_name@4 as n_name, c_address@5 as c_address, c_phone@3 as c_phone, c_comment@6 as c_comment] + │ AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@2 as c_acctbal, c_phone@3 as c_phone, n_name@4 as n_name, c_address@5 as c_address, c_comment@6 as c_comment], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] │ RepartitionExec: partitioning=Hash([c_custkey@0, c_name@1, c_acctbal@2, c_phone@3, n_name@4, c_address@5, c_comment@6], 18), input_partitions=4 @@ -588,22 +598,23 @@ mod tests { │ ProjectionExec: expr=[CAST(CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] │ AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] │ CoalescePartitionsExec - │ [Stage 3] => NetworkCoalesceExec: output_partitions=16, input_tasks=4 + │ [Stage 3] => NetworkCoalesceExec: output_partitions=4, input_tasks=4 │ ProjectionExec: expr=[ps_partkey@0 as ps_partkey, sum(partsupp.ps_supplycost * partsupp.ps_availqty)@1 as sum(partsupp.ps_supplycost * partsupp.ps_availqty), CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@1 AS Decimal128(38, 15)) as join_proj_push_down_1] │ AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] - │ AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[ps_availqty@1, ps_supplycost@2] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ ProjectionExec: expr=[ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost, s_nationkey@0 as s_nationkey] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_nationkey@1, ps_availqty@3, ps_supplycost@4] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/partsupp/1.parquet], [/testdata/tpch/plan_sf0.02/partsupp/10.parquet], [/testdata/tpch/plan_sf0.02/partsupp/11.parquet], [/testdata/tpch/plan_sf0.02/partsupp/12.parquet], [/testdata/tpch/plan_sf0.02/partsupp/13.parquet], ...]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 3 ── Tasks: t0:[p0] t1:[p1] t2:[p2] t3:[p3] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[ps_availqty@1, ps_supplycost@2] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ ProjectionExec: expr=[ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost, s_nationkey@0 as s_nationkey] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_nationkey@1, ps_availqty@3, ps_supplycost@4] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] + │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/partsupp/1.parquet], [/testdata/tpch/plan_sf0.02/partsupp/10.parquet], [/testdata/tpch/plan_sf0.02/partsupp/11.parquet], [/testdata/tpch/plan_sf0.02/partsupp/12.parquet], [/testdata/tpch/plan_sf0.02/partsupp/13.parquet], ...]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 @@ -650,13 +661,14 @@ mod tests { assert_snapshot!(plan, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST] - │ [Stage 3] => NetworkCoalesceExec: output_partitions=18, input_tasks=3 + │ [Stage 3] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ SortExec: expr=[l_shipmode@0 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[l_shipmode@0 as l_shipmode, sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@1 as high_line_count, sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@2 as low_line_count] - │ AggregateExec: mode=FinalPartitioned, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] - │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + ┌───── Stage 3 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST] + │ SortExec: expr=[l_shipmode@0 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[l_shipmode@0 as l_shipmode, sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@1 as high_line_count, sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@2 as low_line_count] + │ AggregateExec: mode=FinalPartitioned, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] │ RepartitionExec: partitioning=Hash([l_shipmode@0], 18), input_partitions=4 @@ -683,13 +695,14 @@ mod tests { assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [custdist@1 DESC, c_count@0 DESC] - │ [Stage 3] => NetworkCoalesceExec: output_partitions=12, input_tasks=2 + │ [Stage 3] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] - │ SortExec: expr=[custdist@1 DESC, c_count@0 DESC], preserve_partitioning=[true] - │ ProjectionExec: expr=[c_count@0 as c_count, count(Int64(1))@1 as custdist] - │ AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(Int64(1))] - │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=3 + ┌───── Stage 3 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [custdist@1 DESC, c_count@0 DESC] + │ SortExec: expr=[custdist@1 DESC, c_count@0 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[c_count@0 as c_count, count(Int64(1))@1 as custdist] + │ AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(Int64(1))] + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] │ RepartitionExec: partitioning=Hash([c_count@0], 12), input_partitions=6 @@ -721,17 +734,18 @@ mod tests { │ ProjectionExec: expr=[100 * CAST(sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END)@0 AS Float64) / CAST(sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 AS Float64) as promo_revenue] │ AggregateExec: mode=Final, gby=[], aggr=[sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=16, input_tasks=4 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=4, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] - │ AggregateExec: mode=Partial, gby=[], aggr=[sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - │ ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as __common_expr_1, p_type@0 as p_type] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], projection=[p_type@1, l_extendedprice@3, l_discount@4] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ FilterExec: l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01, projection=[l_partkey@0, l_extendedprice@1, l_discount@2] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet], [/testdata/tpch/plan_sf0.02/lineitem/10.parquet], [/testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet], [/testdata/tpch/plan_sf0.02/lineitem/13.parquet], ...]}, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@10 >= 1995-09-01 AND l_shipdate@10 < 1995-10-01 AND DynamicFilter [ empty ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1995-09-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1995-10-01, required_guarantees=[] + ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] t2:[p2] t3:[p3] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + │ ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as __common_expr_1, p_type@0 as p_type] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], projection=[p_type@1, l_extendedprice@3, l_discount@4] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ FilterExec: l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01, projection=[l_partkey@0, l_extendedprice@1, l_discount@2] + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] + │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet], [/testdata/tpch/plan_sf0.02/lineitem/10.parquet], [/testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet], [/testdata/tpch/plan_sf0.02/lineitem/13.parquet], ...]}, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@10 >= 1995-09-01 AND l_shipdate@10 < 1995-10-01 AND DynamicFilter [ empty ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1995-09-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1995-10-01, required_guarantees=[] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 @@ -748,31 +762,33 @@ mod tests { assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST] - │ [Stage 6] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=4, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ SortExec: expr=[s_suppkey@0 ASC NULLS LAST], preserve_partitioning=[true] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(max(revenue0.total_revenue)@0, total_revenue@4)], projection=[s_suppkey@1, s_name@2, s_address@3, s_phone@4, total_revenue@5] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=1 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, supplier_no@0)], projection=[s_suppkey@0, s_name@1, s_address@2, s_phone@3, total_revenue@5] + ┌───── Stage 6 ── Tasks: t0:[p0] t1:[p1] t2:[p2] t3:[p3] + │ SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST] + │ SortExec: expr=[s_suppkey@0 ASC NULLS LAST], preserve_partitioning=[true] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(max(revenue0.total_revenue)@0, total_revenue@4)], projection=[s_suppkey@1, s_name@2, s_address@3, s_phone@4, total_revenue@5] │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ ProjectionExec: expr=[l_suppkey@0 as supplier_no, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] - │ AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=1 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, supplier_no@0)], projection=[s_suppkey@0, s_name@1, s_address@2, s_phone@3, total_revenue@5] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ ProjectionExec: expr=[l_suppkey@0 as supplier_no, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] + │ AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p3] │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 │ AggregateExec: mode=Final, gby=[], aggr=[max(revenue0.total_revenue)] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[max(revenue0.total_revenue)] - │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] - │ AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[max(revenue0.total_revenue)] + │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] + │ AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] │ RepartitionExec: partitioning=Hash([l_suppkey@0], 12), input_partitions=4 @@ -803,13 +819,14 @@ mod tests { assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST] - │ [Stage 5] => NetworkCoalesceExec: output_partitions=12, input_tasks=2 + │ [Stage 5] => NetworkCoalesceExec: output_partitions=2, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] - │ SortExec: expr=[supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST], preserve_partitioning=[true] - │ ProjectionExec: expr=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, count(alias1)@3 as supplier_cnt] - │ AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[count(alias1)] - │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=3 + ┌───── Stage 5 ── Tasks: t0:[p0] t1:[p1] + │ SortPreservingMergeExec: [supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST] + │ SortExec: expr=[supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, count(alias1)@3 as supplier_cnt] + │ AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[count(alias1)] + │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] │ RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2], 12), input_partitions=6 @@ -854,16 +871,17 @@ mod tests { │ ProjectionExec: expr=[CAST(sum(lineitem.l_extendedprice)@0 AS Float64) / 7 as avg_yearly] │ AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice)] │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=4, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * avg(lineitem.l_quantity)@1, projection=[l_extendedprice@1] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ ProjectionExec: expr=[CAST(0.2 * CAST(avg(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * avg(lineitem.l_quantity), l_partkey@0 as l_partkey] - │ AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] - │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + ┌───── Stage 4 ── Tasks: t0:[p0] t1:[p1] t2:[p2] t3:[p3] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * avg(lineitem.l_quantity)@1, projection=[l_extendedprice@1] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ ProjectionExec: expr=[CAST(0.2 * CAST(avg(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * avg(lineitem.l_quantity), l_partkey@0 as l_partkey] + │ AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 @@ -896,12 +914,13 @@ mod tests { assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST] - │ [Stage 6] => NetworkCoalesceExec: output_partitions=18, input_tasks=3 + │ [Stage 6] => NetworkCoalesceExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ SortExec: expr=[o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST], preserve_partitioning=[true] - │ AggregateExec: mode=FinalPartitioned, gby=[c_name@0 as c_name, c_custkey@1 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@3 as o_orderdate, o_totalprice@4 as o_totalprice], aggr=[sum(lineitem.l_quantity)] - │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + ┌───── Stage 6 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ SortPreservingMergeExec: [o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST] + │ SortExec: expr=[o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST], preserve_partitioning=[true] + │ AggregateExec: mode=FinalPartitioned, gby=[c_name@0 as c_name, c_custkey@1 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@3 as o_orderdate, o_totalprice@4 as o_totalprice], aggr=[sum(lineitem.l_quantity)] + │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] │ RepartitionExec: partitioning=Hash([c_name@0, c_custkey@1, o_orderkey@2, o_orderdate@3, o_totalprice@4], 18), input_partitions=4 @@ -952,16 +971,17 @@ mod tests { │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@0 as revenue] │ AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=16, input_tasks=4 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=4, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] - │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], filter=p_brand@1 = Brand#12 AND p_container@3 IN (SET) ([SM CASE, SM BOX, SM PACK, SM PKG]) AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN (SET) ([MED BAG, MED BOX, MED PKG, MED PACK]) AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN (SET) ([LG CASE, LG BOX, LG PACK, LG PKG]) AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_extendedprice@6, l_discount@7] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ FilterExec: (l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2) AND (l_shipmode@5 = AIR OR l_shipmode@5 = AIR REG) AND l_shipinstruct@4 = DELIVER IN PERSON, projection=[l_partkey@0, l_quantity@1, l_extendedprice@2, l_discount@3] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet], [/testdata/tpch/plan_sf0.02/lineitem/10.parquet], [/testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet], [/testdata/tpch/plan_sf0.02/lineitem/13.parquet], ...]}, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], file_type=parquet, predicate=(l_quantity@4 >= Some(100),15,2 AND l_quantity@4 <= Some(1100),15,2 OR l_quantity@4 >= Some(1000),15,2 AND l_quantity@4 <= Some(2000),15,2 OR l_quantity@4 >= Some(2000),15,2 AND l_quantity@4 <= Some(3000),15,2) AND (l_shipmode@14 = AIR OR l_shipmode@14 = AIR REG) AND l_shipinstruct@13 = DELIVER IN PERSON AND DynamicFilter [ empty ], pruning_predicate=(l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(100),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(1100),15,2 OR l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(1000),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(2000),15,2 OR l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(2000),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(3000),15,2) AND (l_shipmode_null_count@6 != row_count@2 AND l_shipmode_min@4 <= AIR AND AIR <= l_shipmode_max@5 OR l_shipmode_null_count@6 != row_count@2 AND l_shipmode_min@4 <= AIR REG AND AIR REG <= l_shipmode_max@5) AND l_shipinstruct_null_count@9 != row_count@2 AND l_shipinstruct_min@7 <= DELIVER IN PERSON AND DELIVER IN PERSON <= l_shipinstruct_max@8, required_guarantees=[l_shipinstruct in (DELIVER IN PERSON), l_shipmode in (AIR, AIR REG)] + ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] t2:[p2] t3:[p3] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], filter=p_brand@1 = Brand#12 AND p_container@3 IN (SET) ([SM CASE, SM BOX, SM PACK, SM PKG]) AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN (SET) ([MED BAG, MED BOX, MED PKG, MED PACK]) AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN (SET) ([LG CASE, LG BOX, LG PACK, LG PKG]) AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_extendedprice@6, l_discount@7] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ FilterExec: (l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2) AND (l_shipmode@5 = AIR OR l_shipmode@5 = AIR REG) AND l_shipinstruct@4 = DELIVER IN PERSON, projection=[l_partkey@0, l_quantity@1, l_extendedprice@2, l_discount@3] + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] + │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet], [/testdata/tpch/plan_sf0.02/lineitem/10.parquet], [/testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet], [/testdata/tpch/plan_sf0.02/lineitem/13.parquet], ...]}, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], file_type=parquet, predicate=(l_quantity@4 >= Some(100),15,2 AND l_quantity@4 <= Some(1100),15,2 OR l_quantity@4 >= Some(1000),15,2 AND l_quantity@4 <= Some(2000),15,2 OR l_quantity@4 >= Some(2000),15,2 AND l_quantity@4 <= Some(3000),15,2) AND (l_shipmode@14 = AIR OR l_shipmode@14 = AIR REG) AND l_shipinstruct@13 = DELIVER IN PERSON AND DynamicFilter [ empty ], pruning_predicate=(l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(100),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(1100),15,2 OR l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(1000),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(2000),15,2 OR l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(2000),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(3000),15,2) AND (l_shipmode_null_count@6 != row_count@2 AND l_shipmode_min@4 <= AIR AND AIR <= l_shipmode_max@5 OR l_shipmode_null_count@6 != row_count@2 AND l_shipmode_min@4 <= AIR REG AND AIR REG <= l_shipmode_max@5) AND l_shipinstruct_null_count@9 != row_count@2 AND l_shipinstruct_min@7 <= DELIVER IN PERSON AND DELIVER IN PERSON <= l_shipinstruct_max@8, required_guarantees=[l_shipinstruct in (DELIVER IN PERSON), l_shipmode in (AIR, AIR REG)] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 @@ -982,7 +1002,7 @@ mod tests { │ SortExec: expr=[s_name@0 ASC NULLS LAST], preserve_partitioning=[true] │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_name@1, s_address@2] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=16, input_tasks=4 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=4, input_tasks=4 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * sum(lineitem.l_quantity)@1, projection=[ps_suppkey@1] │ CoalescePartitionsExec │ BroadcastExec: input_partitions=6, consumer_tasks=1, output_partitions=6 @@ -994,12 +1014,13 @@ mod tests { │ AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[s_suppkey@1, s_name@2, s_address@3] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet], [/testdata/tpch/plan_sf0.02/supplier/10.parquet], [/testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet], [/testdata/tpch/plan_sf0.02/supplier/13.parquet], ...]}, projection=[s_suppkey, s_name, s_address, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] t2:[p2] t3:[p3] + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[s_suppkey@1, s_name@2, s_address@3] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] + │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet], [/testdata/tpch/plan_sf0.02/supplier/10.parquet], [/testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet], [/testdata/tpch/plan_sf0.02/supplier/13.parquet], ...]}, projection=[s_suppkey, s_name, s_address, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 @@ -1039,24 +1060,25 @@ mod tests { │ CoalescePartitionsExec │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=16, input_tasks=4 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=4, input_tasks=4 │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_suppkey], file_type=parquet │ FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@12 > l_commitdate@11 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@1)], projection=[s_name@1, l_orderkey@3, l_suppkey@4] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@2)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] + ┌───── Stage 4 ── Tasks: t0:[p0] t1:[p1] t2:[p2] t3:[p3] + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@1)], projection=[s_name@1, l_orderkey@3, l_suppkey@4] │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@2)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet], [/testdata/tpch/plan_sf0.02/lineitem/10.parquet], [/testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet], [/testdata/tpch/plan_sf0.02/lineitem/13.parquet], ...]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@12 > l_commitdate@11 AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] + │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet], [/testdata/tpch/plan_sf0.02/lineitem/10.parquet], [/testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet], [/testdata/tpch/plan_sf0.02/lineitem/13.parquet], ...]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@12 > l_commitdate@11 AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 @@ -1094,23 +1116,25 @@ mod tests { │ NestedLoopJoinExec: join_type=Inner, filter=join_proj_push_down_1@1 > avg(customer.c_acctbal)@0, projection=[avg(customer.c_acctbal)@0, c_phone@1, c_acctbal@2] │ AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)] │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=16, input_tasks=4 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=4, input_tasks=4 │ ProjectionExec: expr=[c_phone@0 as c_phone, c_acctbal@1 as c_acctbal, CAST(c_acctbal@1 AS Decimal128(19, 6)) as join_proj_push_down_1] │ HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)], projection=[c_phone@1, c_acctbal@2] │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=16, input_tasks=4 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=4, input_tasks=4 │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet, /testdata/tpch/plan_sf0.02/orders/10.parquet, /testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet, /testdata/tpch/plan_sf0.02/orders/13.parquet, /testdata/tpch/plan_sf0.02/orders/14.parquet], [/testdata/tpch/plan_sf0.02/orders/15.parquet, /testdata/tpch/plan_sf0.02/orders/16.parquet, /testdata/tpch/plan_sf0.02/orders/2.parquet], [/testdata/tpch/plan_sf0.02/orders/3.parquet, /testdata/tpch/plan_sf0.02/orders/4.parquet, /testdata/tpch/plan_sf0.02/orders/5.parquet], [/testdata/tpch/plan_sf0.02/orders/6.parquet, /testdata/tpch/plan_sf0.02/orders/7.parquet, /testdata/tpch/plan_sf0.02/orders/8.parquet], ...]}, projection=[o_custkey], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] - │ AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] - │ FilterExec: c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]), projection=[c_acctbal@1] - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/customer/1.parquet], [/testdata/tpch/plan_sf0.02/customer/10.parquet], [/testdata/tpch/plan_sf0.02/customer/11.parquet], [/testdata/tpch/plan_sf0.02/customer/12.parquet], [/testdata/tpch/plan_sf0.02/customer/13.parquet], ...]}, projection=[c_phone, c_acctbal], file_type=parquet, predicate=c_acctbal@5 > Some(0),15,2 AND substr(c_phone@4, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]), pruning_predicate=c_acctbal_null_count@1 != row_count@2 AND c_acctbal_max@0 > Some(0),15,2, required_guarantees=[] + ┌───── Stage 1 ── Tasks: t0:[p0] t1:[p1] t2:[p2] t3:[p3] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] + │ FilterExec: c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]), projection=[c_acctbal@1] + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] + │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/customer/1.parquet], [/testdata/tpch/plan_sf0.02/customer/10.parquet], [/testdata/tpch/plan_sf0.02/customer/11.parquet], [/testdata/tpch/plan_sf0.02/customer/12.parquet], [/testdata/tpch/plan_sf0.02/customer/13.parquet], ...]}, projection=[c_phone, c_acctbal], file_type=parquet, predicate=c_acctbal@5 > Some(0),15,2 AND substr(c_phone@4, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]), pruning_predicate=c_acctbal_null_count@1 != row_count@2 AND c_acctbal_max@0 > Some(0),15,2, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] - │ FilterExec: substr(c_phone@1, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]) - │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] - │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/customer/1.parquet], [/testdata/tpch/plan_sf0.02/customer/10.parquet], [/testdata/tpch/plan_sf0.02/customer/11.parquet], [/testdata/tpch/plan_sf0.02/customer/12.parquet], [/testdata/tpch/plan_sf0.02/customer/13.parquet], ...]}, projection=[c_custkey, c_phone, c_acctbal], file_type=parquet, predicate=substr(c_phone@4, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]) + ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] t2:[p2] t3:[p3] + │ CoalescePartitionsExec + │ FilterExec: substr(c_phone@1, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]) + │ PartitionIsolatorExec: t0:[p0,p1,p2,p3,__,__,__,__,__,__,__,__,__,__,__,__] t1:[__,__,__,__,p0,p1,p2,p3,__,__,__,__,__,__,__,__] t2:[__,__,__,__,__,__,__,__,p0,p1,p2,p3,__,__,__,__] t3:[__,__,__,__,__,__,__,__,__,__,__,__,p0,p1,p2,p3] + │ DataSourceExec: file_groups={16 groups: [[/testdata/tpch/plan_sf0.02/customer/1.parquet], [/testdata/tpch/plan_sf0.02/customer/10.parquet], [/testdata/tpch/plan_sf0.02/customer/11.parquet], [/testdata/tpch/plan_sf0.02/customer/12.parquet], [/testdata/tpch/plan_sf0.02/customer/13.parquet], ...]}, projection=[c_custkey, c_phone, c_acctbal], file_type=parquet, predicate=substr(c_phone@4, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]) └────────────────────────────────────────────────── "); Ok(())