diff --git a/src/distributed_planner/distributed_physical_optimizer_rule.rs b/src/distributed_planner/distributed_physical_optimizer_rule.rs index 1f31e171..2e052241 100644 --- a/src/distributed_planner/distributed_physical_optimizer_rule.rs +++ b/src/distributed_planner/distributed_physical_optimizer_rule.rs @@ -719,15 +719,15 @@ mod tests { └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] t2:[p2] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(RainToday@1, RainToday@1)], projection=[MinTemp@0, MaxTemp@2] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ 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=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 - │ 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 + │ CoalescePartitionsExec + │ 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 └────────────────────────────────────────────────── ") } @@ -748,23 +748,23 @@ mod tests { └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0] t1:[p1] t2:[p2] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(RainToday@2, RainToday@1)], projection=[MinTemp@0, MaxTemp@1, Rainfall@3] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ 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=[Rainfall, RainToday], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(RainToday@1, RainToday@1)], projection=[MinTemp@0, MaxTemp@2, RainToday@3] - │ CoalescePartitionsExec + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(RainToday@1, RainToday@1)], projection=[MinTemp@0, MaxTemp@2, RainToday@3] │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 - │ 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=DynamicFilter [ empty ] + │ 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=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 - │ 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 + │ CoalescePartitionsExec + │ 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 └────────────────────────────────────────────────── ") } @@ -793,15 +793,15 @@ mod tests { └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] t2:[p2] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(RainToday@1, RainToday@1)], projection=[MinTemp@0, MaxTemp@2] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ 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=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 - │ 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 + │ CoalescePartitionsExec + │ 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 └────────────────────────────────────────────────── ") } @@ -831,18 +831,18 @@ mod tests { ┌───── 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 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec │ 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 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec │ 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 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec │ 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 ] └────────────────────────────────────────────────── @@ -864,14 +864,14 @@ mod tests { └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] t2:[p2] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(RainToday@1, RainToday@1)], projection=[MinTemp@0, MaxTemp@2] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=1 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=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=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 + ┌───── Stage 1 ── Tasks: t0:[p0..p2] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 └────────────────────────────────────────────────── "); } diff --git a/src/distributed_planner/insert_broadcast.rs b/src/distributed_planner/insert_broadcast.rs index b5b8e941..8b5da70a 100644 --- a/src/distributed_planner/insert_broadcast.rs +++ b/src/distributed_planner/insert_broadcast.rs @@ -5,8 +5,7 @@ use datafusion::common::tree_node::{Transformed, TreeNode}; use datafusion::config::ConfigOptions; use datafusion::error::DataFusionError; use datafusion::physical_plan::ExecutionPlan; -use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; -use datafusion::physical_plan::joins::{HashJoinExec, PartitionMode}; +use datafusion::physical_plan::joins::{HashJoinExec, NestedLoopJoinExec, PartitionMode}; use crate::BroadcastExec; @@ -121,17 +120,19 @@ pub(super) fn insert_broadcast_execs( } plan.transform_down(|node| { - let Some(hash_join) = node.as_any().downcast_ref::() else { + let join_type = if let Some(join) = node.as_any().downcast_ref::() { + join.join_type() + } else if let Some(join) = node.as_any().downcast_ref::() + && join.partition_mode() == &PartitionMode::CollectLeft + { + join.join_type() + } else { return Ok(Transformed::no(node)); }; - if hash_join.partition_mode() != &PartitionMode::CollectLeft { - return Ok(Transformed::no(node)); - } // Only broadcast when output is driven by the probe side. // Joins that can emit build-side rows (left/left-semi/left-anti/left-mark/full) would // duplicate output if the build is broadcast, thus are excluded. - let join_type = hash_join.join_type(); if !matches!( join_type, JoinType::Inner @@ -148,30 +149,15 @@ pub(super) fn insert_broadcast_execs( return Ok(Transformed::no(node)); }; - // If build child is CoalescePartitionsExec get its input - // Otherwise, use the build child directly (DataSourceExec) - let broadcast_input = if let Some(coalesce) = build_child - .as_any() - .downcast_ref::() - { - Arc::clone(coalesce.input()) - } else { - Arc::clone(build_child) - }; - // Insert BroadcastExec. consumer_task_count=1 is a placeholder and // will be corrected during optimizer rule. let broadcast = Arc::new(BroadcastExec::new( - broadcast_input, + Arc::clone(build_child), 1, // placeholder )); - // Always wrap with CoalescePartitionsExec - let new_build_child: Arc = - Arc::new(CoalescePartitionsExec::new(broadcast)); - let mut new_children: Vec> = children.into_iter().cloned().collect(); - new_children[0] = new_build_child; + new_children[0] = broadcast; Ok(Transformed::yes(node.with_new_children(new_children)?)) }) .map(|transformed| transformed.data) @@ -203,8 +189,8 @@ mod tests { let plan = sql_to_plan_with_broadcast(query, true, 4).await; assert_snapshot!(plan, @r" 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 + BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + CoalescePartitionsExec 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 ] "); @@ -226,9 +212,8 @@ mod tests { let plan = sql_to_plan_with_broadcast(query, true, 1).await; assert_snapshot!(plan, @r" HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(RainToday@1, RainToday@1)], projection=[MinTemp@0, MaxTemp@2] - CoalescePartitionsExec - BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 - DataSourceExec: file_groups={1 group: [[/testdata/weather/result-000000.parquet, /testdata/weather/result-000001.parquet, /testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet + BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + DataSourceExec: file_groups={1 group: [[/testdata/weather/result-000000.parquet, /testdata/weather/result-000001.parquet, /testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet DataSourceExec: file_groups={1 group: [[/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/plan_annotator.rs b/src/distributed_planner/plan_annotator.rs index f54b4271..30f44442 100644 --- a/src/distributed_planner/plan_annotator.rs +++ b/src/distributed_planner/plan_annotator.rs @@ -162,12 +162,12 @@ pub(super) fn annotate_plan( plan: Arc, cfg: &ConfigOptions, ) -> Result { - _annotate_plan(plan, None, cfg, true) + _annotate_plan(plan, &mut vec![], cfg, true) } fn _annotate_plan( plan: Arc, - parent: Option<&Arc>, + parent_stack: &mut Vec>, cfg: &ConfigOptions, root: bool, ) -> Result { @@ -176,11 +176,13 @@ fn _annotate_plan( let estimator = &d_cfg.__private_task_estimator; let n_workers = d_cfg.__private_worker_resolver.0.get_urls()?.len().max(1); + parent_stack.push(Arc::clone(&plan)); let annotated_children = plan .children() .iter() - .map(|child| _annotate_plan(Arc::clone(child), Some(&plan), cfg, false)) + .map(|child| _annotate_plan(Arc::clone(child), parent_stack, cfg, false)) .collect::, _>>()?; + parent_stack.pop(); if plan.children().is_empty() { // This is a leaf node, maybe a DataSourceExec, or maybe something else custom from the @@ -255,29 +257,39 @@ fn _annotate_plan( task_count, }; } - } else if let Some(parent) = parent + } else if let Some(_b_exec) = plan.as_any().downcast_ref::() { + annotation = AnnotatedPlan { + plan_or_nb: PlanOrNetworkBoundary::Broadcast, + children: vec![annotation], + task_count, + }; + } else if let Some(parent) = parent_stack.last() // If this node is a leaf node, putting a network boundary above is a bit wasteful, so // we don't want to do it. && !plan.children().is_empty() // If the parent is trying to coalesce all partitions into one, we need to introduce // a network coalesce right below it (or in other words, above the current node) - && (parent.as_any().is::() - || parent.as_any().is::()) + && parent.as_any().is::() + // If the node above the CoalescePartitionsExec or SortPreservingMergeExec is a + // BroadcastExec, then we don't need to insert a network boundary, because we are in the + // situation where a broadcast join will happen. + && !parent_stack.iter().rev().take(2).any(|plan| plan.as_any().is::()) { - // A BroadcastExec underneath a coalesce parent means the build side will cross stages. - if plan.as_any().is::() { - annotation = AnnotatedPlan { - plan_or_nb: PlanOrNetworkBoundary::Broadcast, - children: vec![annotation], - task_count, - }; - } else { - annotation = AnnotatedPlan { - plan_or_nb: PlanOrNetworkBoundary::Coalesce, - children: vec![annotation], - task_count, - }; - } + annotation = AnnotatedPlan { + plan_or_nb: PlanOrNetworkBoundary::Coalesce, + children: vec![annotation], + task_count, + }; + } else if let Some(parent) = parent_stack.last() + // SortPreservingMergeExec also coalesces all partitions into one, we need to introduce + // a network coalesce right below it + && parent.as_any().is::() + { + annotation = AnnotatedPlan { + plan_or_nb: PlanOrNetworkBoundary::Coalesce, + children: vec![annotation], + task_count, + }; } // The plan needs a NetworkBoundary. At this point we have all the info we need for choosing @@ -690,9 +702,9 @@ mod tests { let annotated = sql_to_annotated_broadcast(query, 4, 4, true).await; assert_snapshot!(annotated, @r" HashJoinExec: task_count=Desired(3) - CoalescePartitionsExec: task_count=Desired(3) - [NetworkBoundary] Broadcast: task_count=Desired(3) - BroadcastExec: task_count=Desired(3) + [NetworkBoundary] Broadcast: task_count=Desired(3) + BroadcastExec: task_count=Desired(3) + CoalescePartitionsExec: task_count=Desired(3) DataSourceExec: task_count=Desired(3) DataSourceExec: task_count=Desired(3) ") @@ -720,10 +732,9 @@ mod tests { assert!(annotated.contains("Broadcast")); assert_snapshot!(annotated, @r" HashJoinExec: task_count=Desired(3) - CoalescePartitionsExec: task_count=Desired(3) - [NetworkBoundary] Broadcast: task_count=Desired(3) - BroadcastExec: task_count=Desired(3) - DataSourceExec: task_count=Desired(3) + [NetworkBoundary] Broadcast: task_count=Desired(3) + BroadcastExec: task_count=Desired(3) + DataSourceExec: task_count=Desired(3) DataSourceExec: task_count=Desired(3) "); } @@ -739,9 +750,9 @@ mod tests { sql_to_annotated_broadcast_with_estimator(query, 3, BuildSideOneTaskEstimator).await; assert_snapshot!(annotated, @r" HashJoinExec: task_count=Desired(3) - CoalescePartitionsExec: task_count=Desired(3) - [NetworkBoundary] Broadcast: task_count=Desired(3) - BroadcastExec: task_count=Maximum(1) + [NetworkBoundary] Broadcast: task_count=Desired(3) + BroadcastExec: task_count=Maximum(1) + CoalescePartitionsExec: task_count=Maximum(1) DataSourceExec: task_count=Maximum(1) DataSourceExec: task_count=Desired(3) "); @@ -758,12 +769,12 @@ mod tests { sql_to_annotated_broadcast_with_estimator(query, 3, BroadcastBuildCoalesceMaxEstimator) .await; assert_snapshot!(annotated, @r" - HashJoinExec: task_count=Maximum(1) - CoalescePartitionsExec: task_count=Maximum(1) - [NetworkBoundary] Broadcast: task_count=Maximum(1) - BroadcastExec: task_count=Desired(1) - DataSourceExec: task_count=Desired(1) - DataSourceExec: task_count=Maximum(1) + HashJoinExec: task_count=Desired(3) + [NetworkBoundary] Broadcast: task_count=Desired(3) + BroadcastExec: task_count=Desired(3) + CoalescePartitionsExec: task_count=Desired(3) + DataSourceExec: task_count=Desired(3) + DataSourceExec: task_count=Desired(3) "); } @@ -796,13 +807,13 @@ mod tests { let annotated = sql_to_annotated_broadcast(query, 4, 4, true).await; assert_snapshot!(annotated, @r" HashJoinExec: task_count=Desired(3) - CoalescePartitionsExec: task_count=Desired(3) - [NetworkBoundary] Broadcast: task_count=Desired(3) - BroadcastExec: task_count=Desired(3) + [NetworkBoundary] Broadcast: task_count=Desired(3) + BroadcastExec: task_count=Desired(3) + CoalescePartitionsExec: task_count=Desired(3) HashJoinExec: task_count=Desired(3) - CoalescePartitionsExec: task_count=Desired(3) - [NetworkBoundary] Broadcast: task_count=Desired(3) - BroadcastExec: task_count=Desired(3) + [NetworkBoundary] Broadcast: task_count=Desired(3) + BroadcastExec: task_count=Desired(3) + CoalescePartitionsExec: task_count=Desired(3) DataSourceExec: task_count=Desired(3) DataSourceExec: task_count=Desired(3) DataSourceExec: task_count=Desired(3) @@ -831,21 +842,21 @@ mod tests { assert_snapshot!(annotated, @r" ChildrenIsolatorUnionExec: task_count=Desired(4) HashJoinExec: task_count=Maximum(1) - CoalescePartitionsExec: task_count=Maximum(1) - [NetworkBoundary] Broadcast: task_count=Maximum(1) - BroadcastExec: task_count=Desired(1) + [NetworkBoundary] Broadcast: task_count=Maximum(1) + BroadcastExec: task_count=Desired(1) + CoalescePartitionsExec: task_count=Desired(1) DataSourceExec: task_count=Desired(1) DataSourceExec: task_count=Maximum(1) HashJoinExec: task_count=Maximum(1) - CoalescePartitionsExec: task_count=Maximum(1) - [NetworkBoundary] Broadcast: task_count=Maximum(1) - BroadcastExec: task_count=Desired(1) + [NetworkBoundary] Broadcast: task_count=Maximum(1) + BroadcastExec: task_count=Desired(1) + CoalescePartitionsExec: task_count=Desired(1) DataSourceExec: task_count=Desired(1) DataSourceExec: task_count=Maximum(1) HashJoinExec: task_count=Maximum(2) - CoalescePartitionsExec: task_count=Maximum(2) - [NetworkBoundary] Broadcast: task_count=Maximum(2) - BroadcastExec: task_count=Desired(2) + [NetworkBoundary] Broadcast: task_count=Maximum(2) + BroadcastExec: task_count=Desired(2) + CoalescePartitionsExec: task_count=Desired(2) DataSourceExec: task_count=Desired(2) DataSourceExec: task_count=Maximum(2) "); diff --git a/tests/tpcds_plans_test.rs b/tests/tpcds_plans_test.rs index a59892bb..f59484ed 100644 --- a/tests/tpcds_plans_test.rs +++ b/tests/tpcds_plans_test.rs @@ -28,55 +28,55 @@ mod tests { ┌───── 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 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ctr_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@2)], projection=[ctr_store_sk@1, ctr_total_return@2, c_customer_id@4] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, 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_customer_id], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ctr_store_sk@1)], projection=[ctr_customer_sk@2, ctr_store_sk@3, ctr_total_return@4] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[sr_customer_sk@0 as ctr_customer_sk, sr_store_sk@1 as ctr_store_sk, sum(store_returns.sr_return_amt)@2 as ctr_total_return] - │ AggregateExec: mode=FinalPartitioned, gby=[sr_customer_sk@0 as sr_customer_sk, sr_store_sk@1 as sr_store_sk], aggr=[sum(store_returns.sr_return_amt)] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 5 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ctr_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@2)], projection=[ctr_store_sk@1, ctr_total_return@2, c_customer_id@4] + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, input_tasks=2 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, 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_customer_id], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ctr_store_sk@1)], projection=[ctr_customer_sk@2, ctr_store_sk@3, ctr_total_return@4] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, input_tasks=2 + │ ProjectionExec: expr=[sr_customer_sk@0 as ctr_customer_sk, sr_store_sk@1 as ctr_store_sk, sum(store_returns.sr_return_amt)@2 as ctr_total_return] + │ AggregateExec: mode=FinalPartitioned, gby=[sr_customer_sk@0 as sr_customer_sk, sr_store_sk@1 as sr_store_sk], aggr=[sum(store_returns.sr_return_amt)] + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_state@1 = TN, projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_state@1 = TN, projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([sr_customer_sk@0, sr_store_sk@1], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[sr_customer_sk@0 as sr_customer_sk, sr_store_sk@1 as sr_store_sk], aggr=[sum(store_returns.sr_return_amt)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, sr_returned_date_sk@0)], projection=[sr_customer_sk@3, sr_store_sk@4, sr_return_amt@5] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_returned_date_sk, sr_customer_sk, sr_store_sk, sr_return_amt], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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,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 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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,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 = 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 8 ── Tasks: t0:[p0..p5] t1:[p0..p5] │ RepartitionExec: partitioning=Hash([ctr_store_sk@0], 6), input_partitions=3 @@ -89,18 +89,18 @@ mod tests { │ RepartitionExec: partitioning=Hash([sr_customer_sk@0, sr_store_sk@1], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[sr_customer_sk@0 as sr_customer_sk, sr_store_sk@1 as sr_store_sk], aggr=[sum(store_returns.sr_return_amt)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, sr_returned_date_sk@0)], projection=[sr_customer_sk@3, sr_store_sk@4, sr_return_amt@5] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_returned_date_sk, sr_customer_sk, sr_store_sk, sr_return_amt], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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,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 = 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 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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,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 = 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)] └────────────────────────────────────────────────── "); Ok(()) @@ -118,79 +118,79 @@ mod tests { │ 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 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 - │ ProjectionExec: expr=[d_week_seq@0 as d_week_seq1, sun_sales@1 as sun_sales1, mon_sales@2 as mon_sales1, tue_sales@3 as tue_sales1, wed_sales@4 as wed_sales1, thu_sales@5 as thu_sales1, fri_sales@6 as fri_sales1, sat_sales@7 as sat_sales1, CAST(d_week_seq@0 AS Int64) as CAST(y.d_week_seq1 AS Int64)] - │ 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 1] => 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 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ FilterExec: d_year@1 = 2001, projection=[d_week_seq@0] - │ 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_week_seq, 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 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_week_seq@0 as d_week_seq1, sun_sales@1 as sun_sales1, mon_sales@2 as mon_sales1, tue_sales@3 as tue_sales1, wed_sales@4 as wed_sales1, thu_sales@5 as thu_sales1, fri_sales@6 as fri_sales1, sat_sales@7 as sat_sales1, CAST(d_week_seq@0 AS Int64) as CAST(y.d_week_seq1 AS Int64)] + │ 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] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 3] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ FilterExec: d_year@1 = 2001, projection=[d_week_seq@0] + │ 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_week_seq, 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 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ RepartitionExec: partitioning=Hash([d_week_seq@0], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[d_week_seq@1 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)] │ ProjectionExec: expr=[sales_price@2 as sales_price, d_week_seq@0 as d_week_seq, d_day_name@1 as d_day_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, sold_date_sk@0)], projection=[d_week_seq@1, d_day_name@2, sales_price@5] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=2 │ DistributedUnionExec: t0:[c0(0/2)] t1:[c0(1/2)] t2:[c1(0/2)] t3:[c1(1/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@0 as sold_date_sk, ws_ext_sales_price@23 as sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk@0 as sold_date_sk, cs_ext_sales_price@23 as sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p12..p23] - │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_week_seq@1 as d_week_seq, d_day_name@2 as d_day_name, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_week_seq, d_day_name], file_type=parquet + ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p4..p7] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_week_seq@1 as d_week_seq, d_day_name@2 as d_day_name, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_week_seq, d_day_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ FilterExec: d_year@1 = 2002, projection=[d_week_seq@0] - │ 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_week_seq, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + ┌───── Stage 5 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ FilterExec: d_year@1 = 2002, projection=[d_week_seq@0] + │ 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_week_seq, d_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ RepartitionExec: partitioning=Hash([d_week_seq@0], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[d_week_seq@1 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)] │ ProjectionExec: expr=[sales_price@2 as sales_price, d_week_seq@0 as d_week_seq, d_day_name@1 as d_day_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, sold_date_sk@0)], projection=[d_week_seq@1, d_day_name@2, sales_price@5] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=2 │ DistributedUnionExec: t0:[c0(0/2)] t1:[c0(1/2)] t2:[c1(0/2)] t3:[c1(1/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@0 as sold_date_sk, ws_ext_sales_price@23 as sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk@0 as sold_date_sk, cs_ext_sales_price@23 as sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p11] t1:[p12..p23] - │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_week_seq@1 as d_week_seq, d_day_name@2 as d_day_name, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_week_seq, d_day_name], file_type=parquet + ┌───── Stage 6 ── Tasks: t0:[p0..p3] t1:[p4..p7] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_week_seq@1 as d_week_seq, d_day_name@2 as d_day_name, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_week_seq, d_day_name], file_type=parquet └────────────────────────────────────────────────── "#); Ok(()) @@ -213,28 +213,28 @@ mod tests { │ RepartitionExec: partitioning=Hash([d_year@0, i_brand@1, i_brand_id@2], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[d_year@0 as d_year, i_brand@3 as i_brand, i_brand_id@2 as i_brand_id], aggr=[sum(store_sales.ss_ext_sales_price)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@1, i_item_sk@0)], projection=[d_year@0, ss_ext_sales_price@2, i_brand_id@4, i_brand@5] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ FilterExec: i_manufact_id@3 = 128, projection=[i_item_sk@0, i_brand_id@1, i_brand@2] │ 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_brand_id, i_brand, i_manufact_id], file_type=parquet, predicate=i_manufact_id@13 = 128 AND DynamicFilter [ empty ], pruning_predicate=i_manufact_id_null_count@2 != row_count@3 AND i_manufact_id_min@0 <= 128 AND 128 <= i_manufact_id_max@1, required_guarantees=[i_manufact_id in (128)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(dt.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_ext_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 1] => 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_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(dt.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_ext_sales_price@5] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(dt.d_date_sk AS Float64)] - │ FilterExec: d_moy@2 = 11, projection=[d_date_sk@0, d_year@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_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 11, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1, required_guarantees=[d_moy in (11)] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(dt.d_date_sk AS Float64)] + │ FilterExec: d_moy@2 = 11, projection=[d_date_sk@0, d_year@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_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 11, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1, required_guarantees=[d_moy in (11)] └────────────────────────────────────────────────── "); Ok(()) @@ -250,229 +250,229 @@ mod tests { ┌───── 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 + │ [Stage 20] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 - │ ProjectionExec: expr=[customer_id@1 as customer_id, customer_id@2 as customer_id, customer_first_name@3 as customer_first_name, customer_last_name@4 as customer_last_name, customer_preferred_cust_flag@5 as customer_preferred_cust_flag, year_total@6 as year_total, year_total@7 as year_total, year_total@0 as year_total] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)], projection=[year_total@1, customer_id@2, customer_id@3, customer_first_name@4, customer_last_name@5, customer_preferred_cust_flag@6, year_total@7, year_total@8] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)], filter=CASE WHEN year_total@2 > Some(0),24,6 THEN year_total@3 / year_total@2 END > CASE WHEN year_total@0 > Some(0),24,6 THEN year_total@1 / year_total@0 END, projection=[customer_id@0, customer_id@2, customer_first_name@3, customer_last_name@4, customer_preferred_cust_flag@5, year_total@7, year_total@9] - │ CoalescePartitionsExec - │ [Stage 16] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(catalog_sales.cs_ext_list_price - catalog_sales.cs_ext_wholesale_cost - catalog_sales.cs_ext_discount_amt + catalog_sales.cs_ext_sales_price / Int64(2))@8 as year_total] - │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@7 as d_year], aggr=[sum(catalog_sales.cs_ext_list_price - catalog_sales.cs_ext_wholesale_cost - catalog_sales.cs_ext_discount_amt + catalog_sales.cs_ext_sales_price / Int64(2))], ordering_mode=PartiallySorted([7]) - │ [Stage 19] => 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 - │ 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))@1 as year_total] - │ FilterExec: 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))@1 > Some(0),24,6 - │ ProjectionExec: expr=[c_customer_id@0 as c_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 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))] - │ 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 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 20 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[customer_id@1 as customer_id, customer_id@2 as customer_id, customer_first_name@3 as customer_first_name, customer_last_name@4 as customer_last_name, customer_preferred_cust_flag@5 as customer_preferred_cust_flag, year_total@6 as year_total, year_total@7 as year_total, year_total@0 as year_total] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)], projection=[year_total@1, customer_id@2, customer_id@3, customer_first_name@4, customer_last_name@5, customer_preferred_cust_flag@6, year_total@7, year_total@8] + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)], filter=CASE WHEN year_total@2 > Some(0),24,6 THEN year_total@3 / year_total@2 END > CASE WHEN year_total@0 > Some(0),24,6 THEN year_total@1 / year_total@0 END, projection=[customer_id@0, customer_id@2, customer_first_name@3, customer_last_name@4, customer_preferred_cust_flag@5, year_total@7, year_total@9] + │ [Stage 16] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, input_tasks=2 + │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(catalog_sales.cs_ext_list_price - catalog_sales.cs_ext_wholesale_cost - catalog_sales.cs_ext_discount_amt + catalog_sales.cs_ext_sales_price / Int64(2))@8 as year_total] + │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@7 as d_year], aggr=[sum(catalog_sales.cs_ext_list_price - catalog_sales.cs_ext_wholesale_cost - catalog_sales.cs_ext_discount_amt + catalog_sales.cs_ext_sales_price / Int64(2))], ordering_mode=PartiallySorted([7]) + │ [Stage 19] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ 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))@1 as year_total] + │ FilterExec: 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))@1 > Some(0),24,6 + │ ProjectionExec: expr=[c_customer_id@0 as c_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 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))] + │ 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 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 6), input_partitions=2 │ AggregateExec: mode=Partial, 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@11 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]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, ws_ext_discount_amt@8 as ws_ext_discount_amt, ws_ext_sales_price@9 as ws_ext_sales_price, ws_ext_wholesale_cost@10 as ws_ext_wholesale_cost, ws_ext_list_price@11 as ws_ext_list_price, 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@7)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, c_preferred_cust_flag@6, c_birth_country@7, c_login@8, c_email_address@9, ws_ext_discount_amt@11, ws_ext_sales_price@12, ws_ext_wholesale_cost@13, ws_ext_list_price@14] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, ws_bill_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, ws_sold_date_sk@9, ws_ext_discount_amt@11, ws_ext_sales_price@12, ws_ext_wholesale_cost@13, ws_ext_list_price@14] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_customer_sk, ws_ext_discount_amt, ws_ext_sales_price, ws_ext_wholesale_cost, ws_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001 + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001 + │ 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_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 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, 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/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)] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, 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_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ ProjectionExec: expr=[customer_id@1 as customer_id, year_total@2 as year_total, customer_id@3 as customer_id, customer_first_name@4 as customer_first_name, customer_last_name@5 as customer_last_name, customer_preferred_cust_flag@6 as customer_preferred_cust_flag, year_total@7 as year_total, year_total@0 as year_total] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)], projection=[year_total@1, customer_id@2, year_total@3, customer_id@4, customer_first_name@5, customer_last_name@6, customer_preferred_cust_flag@7, year_total@8] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)] - │ CoalescePartitionsExec - │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[c_customer_id@0 as customer_id, c_first_name@1 as customer_first_name, c_last_name@2 as customer_last_name, c_preferred_cust_flag@3 as customer_preferred_cust_flag, sum(store_sales.ss_ext_list_price - store_sales.ss_ext_wholesale_cost - store_sales.ss_ext_discount_amt + store_sales.ss_ext_sales_price / Int64(2))@8 as year_total] - │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@7 as d_year], aggr=[sum(store_sales.ss_ext_list_price - store_sales.ss_ext_wholesale_cost - store_sales.ss_ext_discount_amt + store_sales.ss_ext_sales_price / Int64(2))], ordering_mode=PartiallySorted([7]) - │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(catalog_sales.cs_ext_list_price - catalog_sales.cs_ext_wholesale_cost - catalog_sales.cs_ext_discount_amt + catalog_sales.cs_ext_sales_price / Int64(2))@1 as year_total] - │ FilterExec: sum(catalog_sales.cs_ext_list_price - catalog_sales.cs_ext_wholesale_cost - catalog_sales.cs_ext_discount_amt + catalog_sales.cs_ext_sales_price / Int64(2))@1 > Some(0),24,6 - │ ProjectionExec: expr=[c_customer_id@0 as c_customer_id, sum(catalog_sales.cs_ext_list_price - catalog_sales.cs_ext_wholesale_cost - catalog_sales.cs_ext_discount_amt + catalog_sales.cs_ext_sales_price / Int64(2))@8 as sum(catalog_sales.cs_ext_list_price - catalog_sales.cs_ext_wholesale_cost - catalog_sales.cs_ext_discount_amt + catalog_sales.cs_ext_sales_price / Int64(2))] - │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@7 as d_year], aggr=[sum(catalog_sales.cs_ext_list_price - catalog_sales.cs_ext_wholesale_cost - catalog_sales.cs_ext_discount_amt + catalog_sales.cs_ext_sales_price / Int64(2))], ordering_mode=PartiallySorted([7]) - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 16 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[customer_id@1 as customer_id, year_total@2 as year_total, customer_id@3 as customer_id, customer_first_name@4 as customer_first_name, customer_last_name@5 as customer_last_name, customer_preferred_cust_flag@6 as customer_preferred_cust_flag, year_total@7 as year_total, year_total@0 as year_total] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)], projection=[year_total@1, customer_id@2, year_total@3, customer_id@4, customer_first_name@5, customer_last_name@6, customer_preferred_cust_flag@7, year_total@8] + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)] + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, input_tasks=2 + │ ProjectionExec: expr=[c_customer_id@0 as customer_id, c_first_name@1 as customer_first_name, c_last_name@2 as customer_last_name, c_preferred_cust_flag@3 as customer_preferred_cust_flag, sum(store_sales.ss_ext_list_price - store_sales.ss_ext_wholesale_cost - store_sales.ss_ext_discount_amt + store_sales.ss_ext_sales_price / Int64(2))@8 as year_total] + │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@7 as d_year], aggr=[sum(store_sales.ss_ext_list_price - store_sales.ss_ext_wholesale_cost - store_sales.ss_ext_discount_amt + store_sales.ss_ext_sales_price / Int64(2))], ordering_mode=PartiallySorted([7]) + │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(catalog_sales.cs_ext_list_price - catalog_sales.cs_ext_wholesale_cost - catalog_sales.cs_ext_discount_amt + catalog_sales.cs_ext_sales_price / Int64(2))@1 as year_total] + │ FilterExec: sum(catalog_sales.cs_ext_list_price - catalog_sales.cs_ext_wholesale_cost - catalog_sales.cs_ext_discount_amt + catalog_sales.cs_ext_sales_price / Int64(2))@1 > Some(0),24,6 + │ ProjectionExec: expr=[c_customer_id@0 as c_customer_id, sum(catalog_sales.cs_ext_list_price - catalog_sales.cs_ext_wholesale_cost - catalog_sales.cs_ext_discount_amt + catalog_sales.cs_ext_sales_price / Int64(2))@8 as sum(catalog_sales.cs_ext_list_price - catalog_sales.cs_ext_wholesale_cost - catalog_sales.cs_ext_discount_amt + catalog_sales.cs_ext_sales_price / Int64(2))] + │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@7 as d_year], aggr=[sum(catalog_sales.cs_ext_list_price - catalog_sales.cs_ext_wholesale_cost - catalog_sales.cs_ext_discount_amt + catalog_sales.cs_ext_sales_price / Int64(2))], ordering_mode=PartiallySorted([7]) + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 6), input_partitions=2 │ AggregateExec: mode=Partial, 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@11 as d_year], aggr=[sum(catalog_sales.cs_ext_list_price - catalog_sales.cs_ext_wholesale_cost - catalog_sales.cs_ext_discount_amt + catalog_sales.cs_ext_sales_price / Int64(2))], ordering_mode=PartiallySorted([7]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, cs_ext_discount_amt@8 as cs_ext_discount_amt, cs_ext_sales_price@9 as cs_ext_sales_price, cs_ext_wholesale_cost@10 as cs_ext_wholesale_cost, cs_ext_list_price@11 as cs_ext_list_price, 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@7)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, c_preferred_cust_flag@6, c_birth_country@7, c_login@8, c_email_address@9, cs_ext_discount_amt@11, cs_ext_sales_price@12, cs_ext_wholesale_cost@13, cs_ext_list_price@14] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, cs_bill_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, cs_sold_date_sk@9, cs_ext_discount_amt@11, cs_ext_sales_price@12, cs_ext_wholesale_cost@13, cs_ext_list_price@14] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_customer_sk, cs_ext_discount_amt, cs_ext_sales_price, cs_ext_wholesale_cost, cs_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001 + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001 + │ 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_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..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, 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/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..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, 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_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(store_sales.ss_ext_list_price - store_sales.ss_ext_wholesale_cost - store_sales.ss_ext_discount_amt + store_sales.ss_ext_sales_price / Int64(2))@1 as year_total] - │ FilterExec: sum(store_sales.ss_ext_list_price - store_sales.ss_ext_wholesale_cost - store_sales.ss_ext_discount_amt + store_sales.ss_ext_sales_price / Int64(2))@1 > Some(0),24,6 - │ ProjectionExec: expr=[c_customer_id@0 as c_customer_id, sum(store_sales.ss_ext_list_price - store_sales.ss_ext_wholesale_cost - store_sales.ss_ext_discount_amt + store_sales.ss_ext_sales_price / Int64(2))@8 as sum(store_sales.ss_ext_list_price - store_sales.ss_ext_wholesale_cost - store_sales.ss_ext_discount_amt + store_sales.ss_ext_sales_price / Int64(2))] - │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@7 as d_year], aggr=[sum(store_sales.ss_ext_list_price - store_sales.ss_ext_wholesale_cost - store_sales.ss_ext_discount_amt + store_sales.ss_ext_sales_price / Int64(2))], ordering_mode=PartiallySorted([7]) - │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 12 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(store_sales.ss_ext_list_price - store_sales.ss_ext_wholesale_cost - store_sales.ss_ext_discount_amt + store_sales.ss_ext_sales_price / Int64(2))@1 as year_total] + │ FilterExec: sum(store_sales.ss_ext_list_price - store_sales.ss_ext_wholesale_cost - store_sales.ss_ext_discount_amt + store_sales.ss_ext_sales_price / Int64(2))@1 > Some(0),24,6 + │ ProjectionExec: expr=[c_customer_id@0 as c_customer_id, sum(store_sales.ss_ext_list_price - store_sales.ss_ext_wholesale_cost - store_sales.ss_ext_discount_amt + store_sales.ss_ext_sales_price / Int64(2))@8 as sum(store_sales.ss_ext_list_price - store_sales.ss_ext_wholesale_cost - store_sales.ss_ext_discount_amt + store_sales.ss_ext_sales_price / Int64(2))] + │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@7 as d_year], aggr=[sum(store_sales.ss_ext_list_price - store_sales.ss_ext_wholesale_cost - store_sales.ss_ext_discount_amt + store_sales.ss_ext_sales_price / Int64(2))], ordering_mode=PartiallySorted([7]) + │ [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_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 6), input_partitions=2 │ AggregateExec: mode=Partial, 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@11 as d_year], aggr=[sum(store_sales.ss_ext_list_price - store_sales.ss_ext_wholesale_cost - store_sales.ss_ext_discount_amt + store_sales.ss_ext_sales_price / Int64(2))], ordering_mode=PartiallySorted([7]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, ss_ext_discount_amt@8 as ss_ext_discount_amt, ss_ext_sales_price@9 as ss_ext_sales_price, ss_ext_wholesale_cost@10 as ss_ext_wholesale_cost, ss_ext_list_price@11 as ss_ext_list_price, 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@7)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, c_preferred_cust_flag@6, c_birth_country@7, c_login@8, c_email_address@9, ss_ext_discount_amt@11, ss_ext_sales_price@12, ss_ext_wholesale_cost@13, ss_ext_list_price@14] - │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, ss_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, ss_sold_date_sk@9, ss_ext_discount_amt@11, ss_ext_sales_price@12, ss_ext_wholesale_cost@13, ss_ext_list_price@14] - │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_customer_sk, ss_ext_discount_amt, ss_ext_sales_price, ss_ext_wholesale_cost, ss_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001 + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001 + │ 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_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 10 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, 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/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 10 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, 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_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 15 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 6), input_partitions=2 │ AggregateExec: mode=Partial, 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@11 as d_year], aggr=[sum(store_sales.ss_ext_list_price - store_sales.ss_ext_wholesale_cost - store_sales.ss_ext_discount_amt + store_sales.ss_ext_sales_price / Int64(2))], ordering_mode=PartiallySorted([7]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, ss_ext_discount_amt@8 as ss_ext_discount_amt, ss_ext_sales_price@9 as ss_ext_sales_price, ss_ext_wholesale_cost@10 as ss_ext_wholesale_cost, ss_ext_list_price@11 as ss_ext_list_price, 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@7)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, c_preferred_cust_flag@6, c_birth_country@7, c_login@8, c_email_address@9, ss_ext_discount_amt@11, ss_ext_sales_price@12, ss_ext_wholesale_cost@13, ss_ext_list_price@14] - │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, ss_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, ss_sold_date_sk@9, ss_ext_discount_amt@11, ss_ext_sales_price@12, ss_ext_wholesale_cost@13, ss_ext_list_price@14] - │ CoalescePartitionsExec - │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_customer_sk, ss_ext_discount_amt, ss_ext_sales_price, ss_ext_wholesale_cost, ss_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2002 + ┌───── Stage 13 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2002 + │ 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_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + └────────────────────────────────────────────────── + ┌───── Stage 14 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, 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/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 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] - └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, 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_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 19 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 6), input_partitions=2 │ AggregateExec: mode=Partial, 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@11 as d_year], aggr=[sum(catalog_sales.cs_ext_list_price - catalog_sales.cs_ext_wholesale_cost - catalog_sales.cs_ext_discount_amt + catalog_sales.cs_ext_sales_price / Int64(2))], ordering_mode=PartiallySorted([7]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, cs_ext_discount_amt@8 as cs_ext_discount_amt, cs_ext_sales_price@9 as cs_ext_sales_price, cs_ext_wholesale_cost@10 as cs_ext_wholesale_cost, cs_ext_list_price@11 as cs_ext_list_price, 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@7)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, c_preferred_cust_flag@6, c_birth_country@7, c_login@8, c_email_address@9, cs_ext_discount_amt@11, cs_ext_sales_price@12, cs_ext_wholesale_cost@13, cs_ext_list_price@14] - │ CoalescePartitionsExec - │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, cs_bill_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, cs_sold_date_sk@9, cs_ext_discount_amt@11, cs_ext_sales_price@12, cs_ext_wholesale_cost@13, cs_ext_list_price@14] - │ CoalescePartitionsExec - │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_customer_sk, cs_ext_discount_amt, cs_ext_sales_price, cs_ext_wholesale_cost, cs_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 17 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2002 + ┌───── Stage 17 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2002 + │ 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_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + └────────────────────────────────────────────────── + ┌───── Stage 18 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, 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/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 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] - └────────────────────────────────────────────────── - ┌───── Stage 18 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, 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_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 23 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 6), input_partitions=2 │ AggregateExec: mode=Partial, 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@11 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]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, ws_ext_discount_amt@8 as ws_ext_discount_amt, ws_ext_sales_price@9 as ws_ext_sales_price, ws_ext_wholesale_cost@10 as ws_ext_wholesale_cost, ws_ext_list_price@11 as ws_ext_list_price, 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@7)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, c_preferred_cust_flag@6, c_birth_country@7, c_login@8, c_email_address@9, ws_ext_discount_amt@11, ws_ext_sales_price@12, ws_ext_wholesale_cost@13, ws_ext_list_price@14] - │ CoalescePartitionsExec - │ [Stage 21] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 21] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, ws_bill_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, ws_sold_date_sk@9, ws_ext_discount_amt@11, ws_ext_sales_price@12, ws_ext_wholesale_cost@13, ws_ext_list_price@14] - │ CoalescePartitionsExec - │ [Stage 22] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 22] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_customer_sk, ws_ext_discount_amt, ws_ext_sales_price, ws_ext_wholesale_cost, ws_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 21 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2002 + ┌───── Stage 21 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2002 + │ 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_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + └────────────────────────────────────────────────── + ┌───── Stage 22 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, 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/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 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] - └────────────────────────────────────────────────── - ┌───── Stage 22 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, 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_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── "); Ok(()) @@ -507,71 +507,69 @@ mod tests { │ AggregateExec: mode=Partial, gby=[s_store_id@4 as s_store_id], aggr=[sum(salesreturns.sales_price), sum(salesreturns.profit), sum(salesreturns.return_amt), sum(salesreturns.net_loss)] │ ProjectionExec: expr=[sales_price@1 as sales_price, profit@2 as profit, return_amt@3 as return_amt, net_loss@4 as net_loss, s_store_id@0 as s_store_id] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, store_sk@0)], projection=[s_store_id@1, sales_price@4, profit@5, return_amt@6, net_loss@7] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, date_sk@1)], projection=[store_sk@2, sales_price@4, profit@5, return_amt@6, net_loss@7] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=2 │ DistributedUnionExec: t0:[c0(0/2)] t1:[c0(1/2)] t2:[c1(0/2)] t3:[c1(1/2)] │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] │ 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_store_sk@7 as store_sk, ss_sold_date_sk@0 as date_sk, ss_ext_sales_price@15 as sales_price, CAST(ss_net_profit@22 AS Decimal128(7, 2)) as profit, Some(0),7,2 as return_amt, Some(0),7,2 as net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] │ 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_store_sk@7 as store_sk, sr_returned_date_sk@0 as date_sk, Some(0),7,2 as sales_price, Some(0),7,2 as profit, sr_return_amt@11 as return_amt, CAST(sr_net_loss@19 AS Decimal128(7, 2)) as net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p8..p15] - │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p11] t1:[p12..p23] - │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 - │ 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_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-06, projection=[d_date_sk@0] - │ 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, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-06, 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-06, required_guarantees=[] + ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p4..p7] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ 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_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-06, projection=[d_date_sk@0] + │ 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, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-06, 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-06, required_guarantees=[] └────────────────────────────────────────────────── ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] │ RepartitionExec: partitioning=Hash([cp_catalog_page_id@0], 3), input_partitions=3 │ AggregateExec: mode=Partial, gby=[cp_catalog_page_id@4 as cp_catalog_page_id], aggr=[sum(salesreturns.sales_price), sum(salesreturns.profit), sum(salesreturns.return_amt), sum(salesreturns.net_loss)] │ ProjectionExec: expr=[sales_price@1 as sales_price, profit@2 as profit, return_amt@3 as return_amt, net_loss@4 as net_loss, cp_catalog_page_id@0 as cp_catalog_page_id] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(catalog_page.cp_catalog_page_sk AS Float64)@2, page_sk@0)], projection=[cp_catalog_page_id@1, sales_price@4, profit@5, return_amt@6, net_loss@7] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, date_sk@1)], projection=[page_sk@2, sales_price@4, profit@5, return_amt@6, net_loss@7] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=2 │ DistributedUnionExec: t0:[c0(0/2)] t1:[c0(1/2)] t2:[c1(0/2)] t3:[c1(1/2)] │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_catalog_page_sk@12 as page_sk, cs_sold_date_sk@0 as date_sk, cs_ext_sales_price@23 as sales_price, cs_net_profit@33 as profit, Some(0),7,2 as return_amt, Some(0),7,2 as net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_catalog_page_sk@12 as page_sk, CAST(cr_returned_date_sk@0 AS Float64) as date_sk, Some(0),7,2 as sales_price, Some(0),7,2 as profit, cr_return_amount@18 as return_amt, cr_net_loss@26 as net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p12..p23] - │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 - │ ProjectionExec: expr=[cp_catalog_page_sk@0 as cp_catalog_page_sk, cp_catalog_page_id@1 as cp_catalog_page_id, CAST(cp_catalog_page_sk@0 AS Float64) as CAST(catalog_page.cp_catalog_page_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/catalog_page/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/catalog_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/catalog_page/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/catalog_page/part-3.parquet]]}, projection=[cp_catalog_page_sk, cp_catalog_page_id], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p11] t1:[p12..p23] - │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 - │ 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_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-06, projection=[d_date_sk@0] + ┌───── Stage 4 ── Tasks: t0:[p0..p3] t1:[p4..p7] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[cp_catalog_page_sk@0 as cp_catalog_page_sk, cp_catalog_page_id@1 as cp_catalog_page_id, CAST(cp_catalog_page_sk@0 AS Float64) as CAST(catalog_page.cp_catalog_page_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, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-06, 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-06, required_guarantees=[] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_page/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/catalog_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/catalog_page/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/catalog_page/part-3.parquet]]}, projection=[cp_catalog_page_sk, cp_catalog_page_id], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p3] t1:[p4..p7] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ 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_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-06, projection=[d_date_sk@0] + │ 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, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-06, 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-06, required_guarantees=[] └────────────────────────────────────────────────── ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ RepartitionExec: partitioning=Hash([web_site_id@0], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[web_site_id@4 as web_site_id], aggr=[sum(salesreturns.sales_price), sum(salesreturns.profit), sum(salesreturns.return_amt), sum(salesreturns.net_loss)] │ ProjectionExec: expr=[sales_price@1 as sales_price, profit@2 as profit, return_amt@3 as return_amt, net_loss@4 as net_loss, web_site_id@0 as web_site_id] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_site.web_site_sk AS Float64)@2, wsr_web_site_sk@0)], projection=[web_site_id@1, sales_price@4, profit@5, return_amt@6, net_loss@7] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=8, input_tasks=2 + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, date_sk@1)], projection=[wsr_web_site_sk@2, sales_price@4, profit@5, return_amt@6, net_loss@7] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=2 │ DistributedUnionExec: t0:[c0(0/2)] t1:[c0(1/2)] t2:[c1(0/2)] t3:[c1(1/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_web_site_sk@13 as wsr_web_site_sk, ws_sold_date_sk@0 as date_sk, ws_ext_sales_price@23 as sales_price, ws_net_profit@33 as profit, Some(0),7,2 as return_amt, Some(0),7,2 as net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] @@ -580,18 +578,20 @@ mod tests { │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=2 │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p7] t1:[p8..p15] - │ BroadcastExec: input_partitions=2, consumer_tasks=4, output_partitions=8 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_site/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-3.parquet]]}, projection=[web_site_sk, web_site_id, CAST(web_site_sk@0 AS Float64) as CAST(web_site.web_site_sk AS Float64)], file_type=parquet + ┌───── Stage 7 ── Tasks: t0:[p0..p3] t1:[p4..p7] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_site/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-3.parquet]]}, projection=[web_site_sk, web_site_id, CAST(web_site_sk@0 AS Float64) as CAST(web_site.web_site_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p11] t1:[p12..p23] - │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 - │ 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_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-06, projection=[d_date_sk@0] - │ 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, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-06, 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-06, required_guarantees=[] + ┌───── Stage 8 ── Tasks: t0:[p0..p3] t1:[p4..p7] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ 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_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-06, projection=[d_date_sk@0] + │ 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, predicate=d_date@2 >= 2000-08-23 AND d_date@2 <= 2000-09-06, 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-06, required_guarantees=[] └────────────────────────────────────────────────── ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p0..p5] │ RepartitionExec: partitioning=Hash([wr_item_sk@1, wr_order_number@2], 6), input_partitions=2 @@ -628,24 +628,22 @@ mod tests { └────────────────────────────────────────────────── ┌───── 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 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ AggregateExec: mode=FinalPartitioned, gby=[d_month_seq@0 as d_month_seq], aggr=[] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + ┌───── Stage 2 ── Tasks: t0:[p0..p2] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ AggregateExec: mode=FinalPartitioned, gby=[d_month_seq@0 as d_month_seq], aggr=[] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ RepartitionExec: partitioning=Hash([d_month_seq@0], 3), input_partitions=3 @@ -655,33 +653,35 @@ 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_month_seq, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 1, 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 <= 1 AND 1 <= d_moy_max@5, required_guarantees=[d_moy in (1), d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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_current_price, i_category], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(c.c_customer_sk AS Float64)@2, ss_customer_sk@2)], projection=[ca_state@0, ss_sold_date_sk@3, ss_item_sk@4] - │ 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_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[ca_state@0 as ca_state, c_customer_sk@1 as c_customer_sk, CAST(c_customer_sk@1 AS Float64) as CAST(c.c_customer_sk AS Float64)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@1)], projection=[ca_state@1, c_customer_sk@2] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, 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/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_current_price, i_category], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(c.c_customer_sk AS Float64)@2, ss_customer_sk@2)], projection=[ca_state@0, ss_sold_date_sk@3, ss_item_sk@4] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ca_state@0 as ca_state, c_customer_sk@1 as c_customer_sk, CAST(c_customer_sk@1 AS Float64) as CAST(c.c_customer_sk AS Float64)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@1)], projection=[ca_state@1, c_customer_sk@2] + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p3] t1:[p4..p7] - │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state], file_type=parquet + ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ RepartitionExec: partitioning=Hash([i_category@0], 3), input_partitions=3 @@ -711,39 +711,39 @@ mod tests { │ RepartitionExec: partitioning=Hash([i_item_id@0], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[i_item_id@4 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)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(promotion.p_promo_sk AS Float64)@1, ss_promo_sk@0)], projection=[ss_quantity@3, ss_list_price@4, ss_sales_price@5, ss_coupon_amt@6, i_item_id@7] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_promo_sk@1, ss_quantity@2, ss_list_price@3, ss_sales_price@4, ss_coupon_amt@5, i_item_id@7] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[p_promo_sk@0 as p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)] - │ FilterExec: p_channel_email@1 = N OR p_channel_event@2 = N, projection=[p_promo_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, p_channel_email, p_channel_event], file_type=parquet, predicate=p_channel_email@9 = N OR p_channel_event@14 = N, pruning_predicate=p_channel_email_null_count@2 != row_count@3 AND p_channel_email_min@0 <= N AND N <= p_channel_email_max@1 OR p_channel_event_null_count@6 != row_count@3 AND p_channel_event_min@4 <= N AND N <= p_channel_event_max@5, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_promo_sk@4, ss_quantity@5, ss_list_price@6, ss_sales_price@7, ss_coupon_amt@8] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, ss_cdemo_sk@2)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_promo_sk@5, ss_quantity@6, ss_list_price@7, ss_sales_price@8, ss_coupon_amt@9] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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,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 = 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 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[p_promo_sk@0 as p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)] + │ FilterExec: p_channel_email@1 = N OR p_channel_event@2 = N, projection=[p_promo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, p_channel_email, p_channel_event], file_type=parquet, predicate=p_channel_email@9 = N OR p_channel_event@14 = N, pruning_predicate=p_channel_email_null_count@2 != row_count@3 AND p_channel_email_min@0 <= N AND N <= p_channel_email_max@1 OR p_channel_event_null_count@6 != row_count@3 AND p_channel_event_min@4 <= N AND N <= p_channel_event_max@5, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ 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_promo_sk@4, ss_quantity@5, ss_list_price@6, ss_sales_price@7, ss_coupon_amt@8] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, ss_cdemo_sk@2)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_promo_sk@5, ss_quantity@6, ss_list_price@7, ss_sales_price@8, ss_coupon_amt@9] + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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,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 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 9), input_partitions=2 @@ -771,15 +771,13 @@ mod tests { │ RepartitionExec: partitioning=Hash([s_store_name@0], 3), input_partitions=3 │ AggregateExec: mode=Partial, gby=[s_store_name@1 as s_store_name], aggr=[sum(store_sales.ss_net_profit)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(substr(store.s_zip,Int64(1),Int64(2))@3, substr(v1.ca_zip,Int64(1),Int64(2))@1)], projection=[ss_net_profit@0, s_store_name@1] - │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec │ ProjectionExec: expr=[ss_net_profit@2 as ss_net_profit, s_store_name@0 as s_store_name, s_zip@1 as s_zip, substr(s_zip@1, 1, 2) as substr(store.s_zip,Int64(1),Int64(2))] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@0)], projection=[s_store_name@1, s_zip@2, ss_net_profit@5] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=2, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=1, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_store_sk@3, ss_net_profit@4] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=3, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=1, input_tasks=2 │ 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_store_sk, ss_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] │ ProjectionExec: expr=[ca_zip@0 as ca_zip, substr(ca_zip@0, 1, 2) as substr(v1.ca_zip,Int64(1),Int64(2))] │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(ca_zip@0, ca_zip@0)], NullsEqual: true @@ -791,18 +789,20 @@ mod tests { │ AggregateExec: mode=FinalPartitioned, gby=[ca_zip@0 as ca_zip], aggr=[count(Int64(1))] │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] - │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] - │ 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_qoy@2 = 2 AND d_year@1 = 1998, projection=[d_date_sk@0] - │ 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_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 2 AND d_year@6 = 1998, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 2 AND 2 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1998 AND 1998 <= d_year_max@5, required_guarantees=[d_qoy in (2), d_year in (1998)] + ┌───── Stage 1 ── Tasks: t0:[p0] t1:[p1] + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0] t1:[p1] + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec + │ 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_qoy@2 = 2 AND d_year@1 = 1998, projection=[d_date_sk@0] + │ 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_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 2 AND d_year@6 = 1998, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 2 AND 2 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1998 AND 1998 <= d_year_max@5, required_guarantees=[d_qoy in (2), d_year in (1998)] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ RepartitionExec: partitioning=Hash([ca_zip@0], 3), input_partitions=3 @@ -817,18 +817,18 @@ mod tests { │ RepartitionExec: partitioning=Hash([ca_zip@0], 3), input_partitions=3 │ AggregateExec: mode=Partial, gby=[ca_zip@0 as ca_zip], aggr=[count(Int64(1))] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@0, ca_address_sk@0)], projection=[ca_zip@2] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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_zip], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ FilterExec: c_preferred_cust_flag@1 = Y, projection=[c_current_addr_sk@0] - │ 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_current_addr_sk, c_preferred_cust_flag], file_type=parquet, predicate=c_preferred_cust_flag@10 = Y, pruning_predicate=c_preferred_cust_flag_null_count@2 != row_count@3 AND c_preferred_cust_flag_min@0 <= Y AND Y <= c_preferred_cust_flag_max@1, required_guarantees=[c_preferred_cust_flag in (Y)] + ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ FilterExec: c_preferred_cust_flag@1 = Y, projection=[c_current_addr_sk@0] + │ 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_current_addr_sk, c_preferred_cust_flag], file_type=parquet, predicate=c_preferred_cust_flag@10 = Y, pruning_predicate=c_preferred_cust_flag_null_count@2 != row_count@3 AND c_preferred_cust_flag_min@0 <= Y AND Y <= c_preferred_cust_flag_max@1, required_guarantees=[c_preferred_cust_flag in (Y)] └────────────────────────────────────────────────── "); Ok(()) @@ -1030,86 +1030,86 @@ mod tests { │ AggregateExec: mode=Partial, 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, cd_dep_count@5 as cd_dep_count, cd_dep_employed_count@6 as cd_dep_employed_count, cd_dep_college_count@7 as cd_dep_college_count], aggr=[count(Int64(1))] │ FilterExec: mark@8 OR mark@9, projection=[cd_gender@0, cd_marital_status@1, cd_education_status@2, cd_purchase_estimate@3, cd_credit_rating@4, cd_dep_count@5, cd_dep_employed_count@6, cd_dep_college_count@7] │ HashJoinExec: mode=CollectLeft, join_type=RightMark, on=[(cs_ship_customer_sk@0, CAST(c.c_customer_sk AS Float64)@10)], projection=[cd_gender@1, cd_marital_status@2, cd_education_status@3, cd_purchase_estimate@4, cd_credit_rating@5, cd_dep_count@6, cd_dep_employed_count@7, cd_dep_college_count@8, mark@9, mark@11] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, cd_gender@1 as cd_gender, cd_marital_status@2 as cd_marital_status, cd_education_status@3 as cd_education_status, cd_purchase_estimate@4 as cd_purchase_estimate, cd_credit_rating@5 as cd_credit_rating, cd_dep_count@6 as cd_dep_count, cd_dep_employed_count@7 as cd_dep_employed_count, cd_dep_college_count@8 as cd_dep_college_count, mark@9 as mark, CAST(c_customer_sk@0 AS Float64) as CAST(c.c_customer_sk AS Float64)] │ HashJoinExec: mode=CollectLeft, join_type=RightMark, on=[(ws_bill_customer_sk@0, CAST(c.c_customer_sk AS Float64)@9)], projection=[c_customer_sk@0, cd_gender@1, cd_marital_status@2, cd_education_status@3, cd_purchase_estimate@4, cd_credit_rating@5, cd_dep_count@6, cd_dep_employed_count@7, cd_dep_college_count@8, mark@10] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, cd_gender@1 as cd_gender, cd_marital_status@2 as cd_marital_status, cd_education_status@3 as cd_education_status, cd_purchase_estimate@4 as cd_purchase_estimate, cd_credit_rating@5 as cd_credit_rating, cd_dep_count@6 as cd_dep_count, cd_dep_employed_count@7 as cd_dep_employed_count, cd_dep_college_count@8 as cd_dep_college_count, CAST(c_customer_sk@0 AS Float64) as CAST(c.c_customer_sk AS Float64)] │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(ss_customer_sk@0, CAST(c.c_customer_sk AS Float64)@9)], projection=[c_customer_sk@0, cd_gender@1, cd_marital_status@2, cd_education_status@3, cd_purchase_estimate@4, cd_credit_rating@5, cd_dep_count@6, cd_dep_employed_count@7, cd_dep_college_count@8] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, cd_gender@1 as cd_gender, cd_marital_status@2 as cd_marital_status, cd_education_status@3 as cd_education_status, cd_purchase_estimate@4 as cd_purchase_estimate, cd_credit_rating@5 as cd_credit_rating, cd_dep_count@6 as cd_dep_count, cd_dep_employed_count@7 as cd_dep_employed_count, cd_dep_college_count@8 as cd_dep_college_count, CAST(c_customer_sk@0 AS Float64) as CAST(c.c_customer_sk AS Float64)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_cdemo_sk@1, CAST(customer_demographics.cd_demo_sk AS Float64)@9)], projection=[c_customer_sk@0, cd_gender@3, cd_marital_status@4, cd_education_status@5, cd_purchase_estimate@6, cd_credit_rating@7, cd_dep_count@8, cd_dep_employed_count@9, cd_dep_college_count@10] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_gender, cd_marital_status, cd_education_status, cd_purchase_estimate, cd_credit_rating, cd_dep_count, cd_dep_employed_count, cd_dep_college_count, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_ship_customer_sk@3] - │ CoalescePartitionsExec - │ [Stage 1] => 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_ship_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 = 2002 AND d_moy@2 >= 1 AND d_moy@2 <= 4, projection=[d_date_sk@0] - │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 2002 AND d_moy@8 >= 1 AND d_moy@8 <= 4, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 AND d_moy_null_count@5 != row_count@3 AND d_moy_max@4 >= 1 AND d_moy_null_count@5 != row_count@3 AND d_moy_min@6 <= 4, required_guarantees=[d_year in (2002)] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_bill_customer_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_date_sk, ws_bill_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 = 2002 AND d_moy@2 >= 1 AND d_moy@2 <= 4, projection=[d_date_sk@0] - │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 2002 AND d_moy@8 >= 1 AND d_moy@8 <= 4, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 AND d_moy_null_count@5 != row_count@3 AND d_moy_max@4 >= 1 AND d_moy_null_count@5 != row_count@3 AND d_moy_min@6 <= 4, required_guarantees=[d_year in (2002)] - └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_customer_sk@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/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 = 2002 AND d_moy@2 >= 1 AND d_moy@2 <= 4, projection=[d_date_sk@0] - │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 2002 AND d_moy@8 >= 1 AND d_moy@8 <= 4, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 AND d_moy_null_count@5 != row_count@3 AND d_moy_max@4 >= 1 AND d_moy_null_count@5 != row_count@3 AND d_moy_min@6 <= 4, required_guarantees=[d_year in (2002)] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_ship_customer_sk@3] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_ship_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 = 2002 AND d_moy@2 >= 1 AND d_moy@2 <= 4, projection=[d_date_sk@0] + │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 2002 AND d_moy@8 >= 1 AND d_moy@8 <= 4, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 AND d_moy_null_count@5 != row_count@3 AND d_moy_max@4 >= 1 AND d_moy_null_count@5 != row_count@3 AND d_moy_min@6 <= 4, required_guarantees=[d_year in (2002)] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@2)], projection=[c_customer_sk@1, c_current_cdemo_sk@2] - │ 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/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_addr_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_bill_customer_sk@3] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_customer_sk], 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 - │ FilterExec: ca_county@1 IN (SET) ([Rush County, Toole County, Jefferson County, Dona Ana County, La Porte County]), projection=[ca_address_sk@0] - │ 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_county], file_type=parquet, predicate=ca_county@7 IN (SET) ([Rush County, Toole County, Jefferson County, Dona Ana County, La Porte County]), pruning_predicate=ca_county_null_count@2 != row_count@3 AND ca_county_min@0 <= Rush County AND Rush County <= ca_county_max@1 OR ca_county_null_count@2 != row_count@3 AND ca_county_min@0 <= Toole County AND Toole County <= ca_county_max@1 OR ca_county_null_count@2 != row_count@3 AND ca_county_min@0 <= Jefferson County AND Jefferson County <= ca_county_max@1 OR ca_county_null_count@2 != row_count@3 AND ca_county_min@0 <= Dona Ana County AND Dona Ana County <= ca_county_max@1 OR ca_county_null_count@2 != row_count@3 AND ca_county_min@0 <= La Porte County AND La Porte County <= ca_county_max@1, required_guarantees=[ca_county in (Dona Ana County, Jefferson County, La Porte County, Rush County, Toole County)] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 = 2002 AND d_moy@2 >= 1 AND d_moy@2 <= 4, projection=[d_date_sk@0] + │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 2002 AND d_moy@8 >= 1 AND d_moy@8 <= 4, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 AND d_moy_null_count@5 != row_count@3 AND d_moy_max@4 >= 1 AND d_moy_null_count@5 != row_count@3 AND d_moy_min@6 <= 4, required_guarantees=[d_year in (2002)] + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_customer_sk@3] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 = 2002 AND d_moy@2 >= 1 AND d_moy@2 <= 4, projection=[d_date_sk@0] + │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 2002 AND d_moy@8 >= 1 AND d_moy@8 <= 4, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 AND d_moy_null_count@5 != row_count@3 AND d_moy_max@4 >= 1 AND d_moy_null_count@5 != row_count@3 AND d_moy_min@6 <= 4, required_guarantees=[d_year in (2002)] + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@2)], projection=[c_customer_sk@1, c_current_cdemo_sk@2] + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_addr_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ FilterExec: ca_county@1 IN (SET) ([Rush County, Toole County, Jefferson County, Dona Ana County, La Porte County]), projection=[ca_address_sk@0] + │ 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_county], file_type=parquet, predicate=ca_county@7 IN (SET) ([Rush County, Toole County, Jefferson County, Dona Ana County, La Porte County]), pruning_predicate=ca_county_null_count@2 != row_count@3 AND ca_county_min@0 <= Rush County AND Rush County <= ca_county_max@1 OR ca_county_null_count@2 != row_count@3 AND ca_county_min@0 <= Toole County AND Toole County <= ca_county_max@1 OR ca_county_null_count@2 != row_count@3 AND ca_county_min@0 <= Jefferson County AND Jefferson County <= ca_county_max@1 OR ca_county_null_count@2 != row_count@3 AND ca_county_min@0 <= Dona Ana County AND Dona Ana County <= ca_county_max@1 OR ca_county_null_count@2 != row_count@3 AND ca_county_min@0 <= La Porte County AND La Porte County <= ca_county_max@1, required_guarantees=[ca_county in (Dona Ana County, Jefferson County, La Porte County, Rush County, Toole County)] └────────────────────────────────────────────────── "); Ok(()) @@ -1125,152 +1125,152 @@ mod tests { ┌───── 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 + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 - │ ProjectionExec: expr=[customer_id@1 as customer_id, year_total@2 as year_total, customer_id@3 as customer_id, customer_first_name@4 as customer_first_name, customer_last_name@5 as customer_last_name, customer_preferred_cust_flag@6 as customer_preferred_cust_flag, year_total@7 as year_total, year_total@0 as year_total] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)], projection=[year_total@1, customer_id@2, year_total@3, customer_id@4, customer_first_name@5, customer_last_name@6, customer_preferred_cust_flag@7, year_total@8] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[c_customer_id@0 as customer_id, c_first_name@1 as customer_first_name, c_last_name@2 as customer_last_name, c_preferred_cust_flag@3 as customer_preferred_cust_flag, sum(store_sales.ss_ext_list_price - store_sales.ss_ext_discount_amt)@8 as year_total] - │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@7 as d_year], aggr=[sum(store_sales.ss_ext_list_price - store_sales.ss_ext_discount_amt)], ordering_mode=PartiallySorted([7]) - │ [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 - │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(web_sales.ws_ext_list_price - web_sales.ws_ext_discount_amt)@1 as year_total] - │ FilterExec: sum(web_sales.ws_ext_list_price - web_sales.ws_ext_discount_amt)@1 > Some(0),18,2 - │ ProjectionExec: expr=[c_customer_id@0 as c_customer_id, sum(web_sales.ws_ext_list_price - web_sales.ws_ext_discount_amt)@8 as sum(web_sales.ws_ext_list_price - web_sales.ws_ext_discount_amt)] - │ 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 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 12 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[customer_id@1 as customer_id, year_total@2 as year_total, customer_id@3 as customer_id, customer_first_name@4 as customer_first_name, customer_last_name@5 as customer_last_name, customer_preferred_cust_flag@6 as customer_preferred_cust_flag, year_total@7 as year_total, year_total@0 as year_total] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)], projection=[year_total@1, customer_id@2, year_total@3, customer_id@4, customer_first_name@5, customer_last_name@6, customer_preferred_cust_flag@7, year_total@8] + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)] + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, input_tasks=2 + │ ProjectionExec: expr=[c_customer_id@0 as customer_id, c_first_name@1 as customer_first_name, c_last_name@2 as customer_last_name, c_preferred_cust_flag@3 as customer_preferred_cust_flag, sum(store_sales.ss_ext_list_price - store_sales.ss_ext_discount_amt)@8 as year_total] + │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@7 as d_year], aggr=[sum(store_sales.ss_ext_list_price - store_sales.ss_ext_discount_amt)], ordering_mode=PartiallySorted([7]) + │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(web_sales.ws_ext_list_price - web_sales.ws_ext_discount_amt)@1 as year_total] + │ FilterExec: sum(web_sales.ws_ext_list_price - web_sales.ws_ext_discount_amt)@1 > Some(0),18,2 + │ ProjectionExec: expr=[c_customer_id@0 as c_customer_id, sum(web_sales.ws_ext_list_price - web_sales.ws_ext_discount_amt)@8 as sum(web_sales.ws_ext_list_price - web_sales.ws_ext_discount_amt)] + │ 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 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 6), input_partitions=2 │ AggregateExec: mode=Partial, 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@9 as d_year], aggr=[sum(web_sales.ws_ext_list_price - web_sales.ws_ext_discount_amt)], ordering_mode=PartiallySorted([7]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, ws_ext_discount_amt@8 as ws_ext_discount_amt, ws_ext_list_price@9 as ws_ext_list_price, 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@7)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, c_preferred_cust_flag@6, c_birth_country@7, c_login@8, c_email_address@9, ws_ext_discount_amt@11, ws_ext_list_price@12] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, ws_bill_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, ws_sold_date_sk@9, ws_ext_discount_amt@11, ws_ext_list_price@12] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_customer_sk, ws_ext_discount_amt, ws_ext_list_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 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001 + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001 + │ 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_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 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, 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/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 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, 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_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(store_sales.ss_ext_list_price - store_sales.ss_ext_discount_amt)@1 as year_total] - │ FilterExec: sum(store_sales.ss_ext_list_price - store_sales.ss_ext_discount_amt)@1 > Some(0),18,2 - │ ProjectionExec: expr=[c_customer_id@0 as c_customer_id, sum(store_sales.ss_ext_list_price - store_sales.ss_ext_discount_amt)@8 as sum(store_sales.ss_ext_list_price - store_sales.ss_ext_discount_amt)] - │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@7 as d_year], aggr=[sum(store_sales.ss_ext_list_price - store_sales.ss_ext_discount_amt)], ordering_mode=PartiallySorted([7]) - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 8 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(store_sales.ss_ext_list_price - store_sales.ss_ext_discount_amt)@1 as year_total] + │ FilterExec: sum(store_sales.ss_ext_list_price - store_sales.ss_ext_discount_amt)@1 > Some(0),18,2 + │ ProjectionExec: expr=[c_customer_id@0 as c_customer_id, sum(store_sales.ss_ext_list_price - store_sales.ss_ext_discount_amt)@8 as sum(store_sales.ss_ext_list_price - store_sales.ss_ext_discount_amt)] + │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, c_preferred_cust_flag@3 as c_preferred_cust_flag, c_birth_country@4 as c_birth_country, c_login@5 as c_login, c_email_address@6 as c_email_address, d_year@7 as d_year], aggr=[sum(store_sales.ss_ext_list_price - store_sales.ss_ext_discount_amt)], ordering_mode=PartiallySorted([7]) + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 6), input_partitions=2 │ AggregateExec: mode=Partial, 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@9 as d_year], aggr=[sum(store_sales.ss_ext_list_price - store_sales.ss_ext_discount_amt)], ordering_mode=PartiallySorted([7]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, ss_ext_discount_amt@8 as ss_ext_discount_amt, ss_ext_list_price@9 as ss_ext_list_price, 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@7)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, c_preferred_cust_flag@6, c_birth_country@7, c_login@8, c_email_address@9, ss_ext_discount_amt@11, ss_ext_list_price@12] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, ss_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, ss_sold_date_sk@9, ss_ext_discount_amt@11, ss_ext_list_price@12] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_customer_sk, ss_ext_discount_amt, ss_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001 + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001 + │ 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_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..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, 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/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..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, 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_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 6), input_partitions=2 │ AggregateExec: mode=Partial, 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@9 as d_year], aggr=[sum(store_sales.ss_ext_list_price - store_sales.ss_ext_discount_amt)], ordering_mode=PartiallySorted([7]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, ss_ext_discount_amt@8 as ss_ext_discount_amt, ss_ext_list_price@9 as ss_ext_list_price, 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@7)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, c_preferred_cust_flag@6, c_birth_country@7, c_login@8, c_email_address@9, ss_ext_discount_amt@11, ss_ext_list_price@12] - │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, ss_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, ss_sold_date_sk@9, ss_ext_discount_amt@11, ss_ext_list_price@12] - │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_customer_sk, ss_ext_discount_amt, ss_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2002 + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2002 + │ 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_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + └────────────────────────────────────────────────── + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, 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/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 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] - └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, 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_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 15 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, c_preferred_cust_flag@3, c_birth_country@4, c_login@5, c_email_address@6, d_year@7], 6), input_partitions=2 │ AggregateExec: mode=Partial, 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@9 as d_year], aggr=[sum(web_sales.ws_ext_list_price - web_sales.ws_ext_discount_amt)], ordering_mode=PartiallySorted([7]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, ws_ext_discount_amt@8 as ws_ext_discount_amt, ws_ext_list_price@9 as ws_ext_list_price, 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@7)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, c_preferred_cust_flag@6, c_birth_country@7, c_login@8, c_email_address@9, ws_ext_discount_amt@11, ws_ext_list_price@12] - │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@8, ws_bill_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, c_preferred_cust_flag@4, c_birth_country@5, c_login@6, c_email_address@7, ws_sold_date_sk@9, ws_ext_discount_amt@11, ws_ext_list_price@12] - │ CoalescePartitionsExec - │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_customer_sk, ws_ext_discount_amt, ws_ext_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2002 + ┌───── Stage 13 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2002 + │ 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_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + └────────────────────────────────────────────────── + ┌───── Stage 14 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, 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/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 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] - └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, 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, c_birth_country@5 as c_birth_country, c_login@6 as c_login, c_email_address@7 as c_email_address, 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_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address], file_type=parquet └────────────────────────────────────────────────── "); Ok(()) @@ -1296,29 +1296,29 @@ mod tests { │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, i_category@2, i_class@3, i_current_price@4], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id, i_item_desc@2 as i_item_desc, i_category@5 as i_category, i_class@4 as i_class, i_current_price@3 as i_current_price], aggr=[sum(web_sales.ws_ext_sales_price)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_ext_sales_price@3, i_item_id@4, i_item_desc@5, i_current_price@6, i_class@7, i_category@8] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[ws_sold_date_sk@5 as ws_sold_date_sk, ws_ext_sales_price@6 as ws_ext_sales_price, i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_current_price@2 as i_current_price, i_class@3 as i_class, i_category@4 as i_category] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_item_id@1, i_item_desc@2, i_current_price@3, i_class@4, i_category@5, ws_sold_date_sk@6, ws_ext_sales_price@8] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_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 - │ 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_date@1 >= 1999-02-22 AND d_date@1 <= 1999-03-24, projection=[d_date_sk@0] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_date@1 >= 1999-02-22 AND d_date@1 <= 1999-03-24, projection=[d_date_sk@0] + │ 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, predicate=d_date@2 >= 1999-02-22 AND d_date@2 <= 1999-03-24, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 1999-02-22 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 1999-03-24, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: i_category@5 = Sports OR i_category@5 = Books OR i_category@5 = Home │ 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, predicate=d_date@2 >= 1999-02-22 AND d_date@2 <= 1999-03-24, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 1999-02-22 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 1999-03-24, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_category@5 = Sports OR i_category@5 = Books OR i_category@5 = Home - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id, i_item_desc, i_current_price, i_class, i_category], file_type=parquet, predicate=i_category@12 = Sports OR i_category@12 = Books OR i_category@12 = Home, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Sports AND Sports <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Home AND Home <= i_category_max@1, required_guarantees=[i_category in (Books, Home, Sports)] + │ 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, i_item_desc, i_current_price, i_class, i_category], file_type=parquet, predicate=i_category@12 = Sports OR i_category@12 = Books OR i_category@12 = Home, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Sports AND Sports <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Home AND Home <= i_category_max@1, required_guarantees=[i_category in (Books, Home, Sports)] └────────────────────────────────────────────────── "#); Ok(()) @@ -1336,61 +1336,61 @@ mod tests { ┌───── 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 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@2)], filter=(ca_state@1 = TX OR ca_state@1 = OH) AND ss_net_profit@0 >= Some(10000),6,2 AND ss_net_profit@0 <= Some(20000),6,2 OR (ca_state@1 = OR OR ca_state@1 = NM OR ca_state@1 = KY) AND ss_net_profit@0 >= Some(15000),6,2 AND ss_net_profit@0 <= Some(30000),6,2 OR (ca_state@1 = VA OR ca_state@1 = TX OR ca_state@1 = MS) AND ss_net_profit@0 >= Some(5000),6,2 AND ss_net_profit@0 <= Some(25000),6,2, projection=[ss_sold_date_sk@0, ss_quantity@2, ss_ext_sales_price@3, ss_ext_wholesale_cost@4] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_state@1 as ca_state, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ FilterExec: __common_expr_4@0 AND __common_expr_4@0 AND ca_country@3 = United States AND ca_state@2 IN (SET) ([TX, OH, OR, NM, KY, VA, MS]), projection=[ca_address_sk@1, ca_state@2] - │ ProjectionExec: expr=[__common_expr_5@0 OR ca_state@2 = OH OR ca_state@2 = OR OR ca_state@2 = NM OR ca_state@2 = KY OR ca_state@2 = VA OR __common_expr_5@0 OR ca_state@2 = MS as __common_expr_4, ca_address_sk@1 as ca_address_sk, ca_state@2 as ca_state, ca_country@3 as ca_country] - │ ProjectionExec: expr=[ca_state@1 = TX as __common_expr_5, ca_address_sk@0 as ca_address_sk, ca_state@1 as ca_state, ca_country@2 as ca_country] - │ 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/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state, ca_country], file_type=parquet, predicate=ca_country@10 = United States AND ca_state@8 IN (SET) ([TX, OH, OR, NM, KY, VA, MS]), pruning_predicate=ca_country_null_count@2 != row_count@3 AND ca_country_min@0 <= United States AND United States <= ca_country_max@1 AND (ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= TX AND TX <= ca_state_max@5 OR ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= OH AND OH <= ca_state_max@5 OR ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= OR AND OR <= ca_state_max@5 OR ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= NM AND NM <= ca_state_max@5 OR ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= KY AND KY <= ca_state_max@5 OR ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= VA AND VA <= ca_state_max@5 OR ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= MS AND MS <= ca_state_max@5), required_guarantees=[ca_country in (United States), ca_state in (KY, MS, NM, OH, OR, TX, VA)] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_hdemo_sk@1, CAST(household_demographics.hd_demo_sk AS Float64)@2)], filter=cd_marital_status@1 = M AND cd_education_status@2 = Advanced Degree AND ss_sales_price@0 >= Some(10000),5,2 AND ss_sales_price@0 <= Some(15000),5,2 AND hd_dep_count@3 = 3 OR cd_marital_status@1 = S AND cd_education_status@2 = College AND ss_sales_price@0 >= Some(5000),5,2 AND ss_sales_price@0 <= Some(10000),5,2 AND hd_dep_count@3 = 1 OR cd_marital_status@1 = W AND cd_education_status@2 = 2 yr Degree AND ss_sales_price@0 >= Some(15000),5,2 AND ss_sales_price@0 <= Some(20000),5,2 AND hd_dep_count@3 = 1, projection=[ss_sold_date_sk@0, ss_addr_sk@2, ss_quantity@3, ss_ext_sales_price@5, ss_ext_wholesale_cost@6, ss_net_profit@7] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, hd_dep_count@1 as hd_dep_count, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ FilterExec: (__common_expr_3@0 OR hd_dep_count@2 = 1) AND __common_expr_3@0, projection=[hd_demo_sk@1, hd_dep_count@2] - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ 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_dep_count@3 = 3 OR hd_dep_count@3 = 1 as __common_expr_3, hd_demo_sk, hd_dep_count], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_cdemo_sk@1, CAST(customer_demographics.cd_demo_sk AS Float64)@3)], filter=cd_marital_status@1 = M AND cd_education_status@2 = Advanced Degree AND ss_sales_price@0 >= Some(10000),5,2 AND ss_sales_price@0 <= Some(15000),5,2 OR cd_marital_status@1 = S AND cd_education_status@2 = College AND ss_sales_price@0 >= Some(5000),5,2 AND ss_sales_price@0 <= Some(10000),5,2 OR cd_marital_status@1 = W AND cd_education_status@2 = 2 yr Degree AND ss_sales_price@0 >= Some(15000),5,2 AND ss_sales_price@0 <= Some(20000),5,2, projection=[ss_sold_date_sk@0, ss_hdemo_sk@2, ss_addr_sk@3, ss_quantity@4, ss_sales_price@5, ss_ext_sales_price@6, ss_ext_wholesale_cost@7, ss_net_profit@8, cd_marital_status@10, cd_education_status@11] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] - │ FilterExec: cd_marital_status@1 = M AND cd_education_status@2 = Advanced Degree OR cd_marital_status@1 = S AND cd_education_status@2 = College OR cd_marital_status@1 = W AND cd_education_status@2 = 2 yr Degree - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_marital_status@2 = M AND cd_education_status@3 = Advanced Degree OR cd_marital_status@2 = S AND cd_education_status@3 = College OR cd_marital_status@2 = W AND cd_education_status@3 = 2 yr Degree, pruning_predicate=cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= M AND M <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= Advanced Degree AND Advanced Degree <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= S AND S <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= College AND College <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= W AND W <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= 2 yr Degree AND 2 yr Degree <= cd_education_status_max@5, required_guarantees=[cd_education_status in (2 yr Degree, Advanced Degree, College), cd_marital_status in (M, S, W)] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@4)], projection=[ss_sold_date_sk@2, ss_cdemo_sk@3, ss_hdemo_sk@4, ss_addr_sk@5, ss_quantity@7, ss_sales_price@8, ss_ext_sales_price@9, ss_ext_wholesale_cost@10, ss_net_profit@11] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ FilterExec: (ss_net_profit@9 >= Some(10000),6,2 AND ss_net_profit@9 <= Some(20000),6,2 OR ss_net_profit@9 >= Some(15000),6,2 AND ss_net_profit@9 <= Some(30000),6,2 OR ss_net_profit@9 >= Some(5000),6,2 AND ss_net_profit@9 <= Some(25000),6,2) AND (ss_sales_price@6 >= Some(10000),5,2 AND ss_sales_price@6 <= Some(15000),5,2 OR ss_sales_price@6 >= Some(5000),5,2 AND ss_sales_price@6 <= Some(10000),5,2 OR ss_sales_price@6 >= Some(15000),5,2 AND ss_sales_price@6 <= Some(20000),5,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_cdemo_sk, ss_hdemo_sk, ss_addr_sk, ss_store_sk, ss_quantity, ss_sales_price, ss_ext_sales_price, ss_ext_wholesale_cost, ss_net_profit], file_type=parquet, predicate=(ss_net_profit@22 >= Some(10000),6,2 AND ss_net_profit@22 <= Some(20000),6,2 OR ss_net_profit@22 >= Some(15000),6,2 AND ss_net_profit@22 <= Some(30000),6,2 OR ss_net_profit@22 >= Some(5000),6,2 AND ss_net_profit@22 <= Some(25000),6,2) AND (ss_sales_price@13 >= Some(10000),5,2 AND ss_sales_price@13 <= Some(15000),5,2 OR ss_sales_price@13 >= Some(5000),5,2 AND ss_sales_price@13 <= Some(10000),5,2 OR ss_sales_price@13 >= Some(15000),5,2 AND ss_sales_price@13 <= Some(20000),5,2) AND DynamicFilter [ empty ], pruning_predicate=(ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_max@0 >= Some(10000),6,2 AND ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_min@3 <= Some(20000),6,2 OR ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_max@0 >= Some(15000),6,2 AND ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_min@3 <= Some(30000),6,2 OR ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_max@0 >= Some(5000),6,2 AND ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_min@3 <= Some(25000),6,2) AND (ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_max@4 >= Some(10000),5,2 AND ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_min@6 <= Some(15000),5,2 OR ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_max@4 >= Some(5000),5,2 AND ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_min@6 <= Some(10000),5,2 OR ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_max@4 >= Some(15000),5,2 AND ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_min@6 <= Some(20000),5,2), required_guarantees=[] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@2)], filter=(ca_state@1 = TX OR ca_state@1 = OH) AND ss_net_profit@0 >= Some(10000),6,2 AND ss_net_profit@0 <= Some(20000),6,2 OR (ca_state@1 = OR OR ca_state@1 = NM OR ca_state@1 = KY) AND ss_net_profit@0 >= Some(15000),6,2 AND ss_net_profit@0 <= Some(30000),6,2 OR (ca_state@1 = VA OR ca_state@1 = TX OR ca_state@1 = MS) AND ss_net_profit@0 >= Some(5000),6,2 AND ss_net_profit@0 <= Some(25000),6,2, projection=[ss_sold_date_sk@0, ss_quantity@2, ss_ext_sales_price@3, ss_ext_wholesale_cost@4] + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_state@1 as ca_state, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] + │ FilterExec: __common_expr_4@0 AND __common_expr_4@0 AND ca_country@3 = United States AND ca_state@2 IN (SET) ([TX, OH, OR, NM, KY, VA, MS]), projection=[ca_address_sk@1, ca_state@2] + │ ProjectionExec: expr=[__common_expr_5@0 OR ca_state@2 = OH OR ca_state@2 = OR OR ca_state@2 = NM OR ca_state@2 = KY OR ca_state@2 = VA OR __common_expr_5@0 OR ca_state@2 = MS as __common_expr_4, ca_address_sk@1 as ca_address_sk, ca_state@2 as ca_state, ca_country@3 as ca_country] + │ ProjectionExec: expr=[ca_state@1 = TX as __common_expr_5, ca_address_sk@0 as ca_address_sk, ca_state@1 as ca_state, ca_country@2 as ca_country] + │ 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/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state, ca_country], file_type=parquet, predicate=ca_country@10 = United States AND ca_state@8 IN (SET) ([TX, OH, OR, NM, KY, VA, MS]), pruning_predicate=ca_country_null_count@2 != row_count@3 AND ca_country_min@0 <= United States AND United States <= ca_country_max@1 AND (ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= TX AND TX <= ca_state_max@5 OR ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= OH AND OH <= ca_state_max@5 OR ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= OR AND OR <= ca_state_max@5 OR ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= NM AND NM <= ca_state_max@5 OR ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= KY AND KY <= ca_state_max@5 OR ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= VA AND VA <= ca_state_max@5 OR ca_state_null_count@6 != row_count@3 AND ca_state_min@4 <= MS AND MS <= ca_state_max@5), required_guarantees=[ca_country in (United States), ca_state in (KY, MS, NM, OH, OR, TX, VA)] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_hdemo_sk@1, CAST(household_demographics.hd_demo_sk AS Float64)@2)], filter=cd_marital_status@1 = M AND cd_education_status@2 = Advanced Degree AND ss_sales_price@0 >= Some(10000),5,2 AND ss_sales_price@0 <= Some(15000),5,2 AND hd_dep_count@3 = 3 OR cd_marital_status@1 = S AND cd_education_status@2 = College AND ss_sales_price@0 >= Some(5000),5,2 AND ss_sales_price@0 <= Some(10000),5,2 AND hd_dep_count@3 = 1 OR cd_marital_status@1 = W AND cd_education_status@2 = 2 yr Degree AND ss_sales_price@0 >= Some(15000),5,2 AND ss_sales_price@0 <= Some(20000),5,2 AND hd_dep_count@3 = 1, projection=[ss_sold_date_sk@0, ss_addr_sk@2, ss_quantity@3, ss_ext_sales_price@5, ss_ext_wholesale_cost@6, ss_net_profit@7] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, hd_dep_count@1 as hd_dep_count, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ FilterExec: (__common_expr_3@0 OR hd_dep_count@2 = 1) AND __common_expr_3@0, projection=[hd_demo_sk@1, hd_dep_count@2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ 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_dep_count@3 = 3 OR hd_dep_count@3 = 1 as __common_expr_3, hd_demo_sk, hd_dep_count], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_cdemo_sk@1, CAST(customer_demographics.cd_demo_sk AS Float64)@3)], filter=cd_marital_status@1 = M AND cd_education_status@2 = Advanced Degree AND ss_sales_price@0 >= Some(10000),5,2 AND ss_sales_price@0 <= Some(15000),5,2 OR cd_marital_status@1 = S AND cd_education_status@2 = College AND ss_sales_price@0 >= Some(5000),5,2 AND ss_sales_price@0 <= Some(10000),5,2 OR cd_marital_status@1 = W AND cd_education_status@2 = 2 yr Degree AND ss_sales_price@0 >= Some(15000),5,2 AND ss_sales_price@0 <= Some(20000),5,2, projection=[ss_sold_date_sk@0, ss_hdemo_sk@2, ss_addr_sk@3, ss_quantity@4, ss_sales_price@5, ss_ext_sales_price@6, ss_ext_wholesale_cost@7, ss_net_profit@8, cd_marital_status@10, cd_education_status@11] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] + │ FilterExec: cd_marital_status@1 = M AND cd_education_status@2 = Advanced Degree OR cd_marital_status@1 = S AND cd_education_status@2 = College OR cd_marital_status@1 = W AND cd_education_status@2 = 2 yr Degree + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_marital_status@2 = M AND cd_education_status@3 = Advanced Degree OR cd_marital_status@2 = S AND cd_education_status@3 = College OR cd_marital_status@2 = W AND cd_education_status@3 = 2 yr Degree, pruning_predicate=cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= M AND M <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= Advanced Degree AND Advanced Degree <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= S AND S <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= College AND College <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= W AND W <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= 2 yr Degree AND 2 yr Degree <= cd_education_status_max@5, required_guarantees=[cd_education_status in (2 yr Degree, Advanced Degree, College), cd_marital_status in (M, S, W)] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@4)], projection=[ss_sold_date_sk@2, ss_cdemo_sk@3, ss_hdemo_sk@4, ss_addr_sk@5, ss_quantity@7, ss_sales_price@8, ss_ext_sales_price@9, ss_ext_wholesale_cost@10, ss_net_profit@11] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ FilterExec: (ss_net_profit@9 >= Some(10000),6,2 AND ss_net_profit@9 <= Some(20000),6,2 OR ss_net_profit@9 >= Some(15000),6,2 AND ss_net_profit@9 <= Some(30000),6,2 OR ss_net_profit@9 >= Some(5000),6,2 AND ss_net_profit@9 <= Some(25000),6,2) AND (ss_sales_price@6 >= Some(10000),5,2 AND ss_sales_price@6 <= Some(15000),5,2 OR ss_sales_price@6 >= Some(5000),5,2 AND ss_sales_price@6 <= Some(10000),5,2 OR ss_sales_price@6 >= Some(15000),5,2 AND ss_sales_price@6 <= Some(20000),5,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_cdemo_sk, ss_hdemo_sk, ss_addr_sk, ss_store_sk, ss_quantity, ss_sales_price, ss_ext_sales_price, ss_ext_wholesale_cost, ss_net_profit], file_type=parquet, predicate=(ss_net_profit@22 >= Some(10000),6,2 AND ss_net_profit@22 <= Some(20000),6,2 OR ss_net_profit@22 >= Some(15000),6,2 AND ss_net_profit@22 <= Some(30000),6,2 OR ss_net_profit@22 >= Some(5000),6,2 AND ss_net_profit@22 <= Some(25000),6,2) AND (ss_sales_price@13 >= Some(10000),5,2 AND ss_sales_price@13 <= Some(15000),5,2 OR ss_sales_price@13 >= Some(5000),5,2 AND ss_sales_price@13 <= Some(10000),5,2 OR ss_sales_price@13 >= Some(15000),5,2 AND ss_sales_price@13 <= Some(20000),5,2) AND DynamicFilter [ empty ], pruning_predicate=(ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_max@0 >= Some(10000),6,2 AND ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_min@3 <= Some(20000),6,2 OR ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_max@0 >= Some(15000),6,2 AND ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_min@3 <= Some(30000),6,2 OR ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_max@0 >= Some(5000),6,2 AND ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_min@3 <= Some(25000),6,2) AND (ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_max@4 >= Some(10000),5,2 AND ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_min@6 <= Some(15000),5,2 OR ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_max@4 >= Some(5000),5,2 AND ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_min@6 <= Some(10000),5,2 OR ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_max@4 >= Some(15000),5,2 AND ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_min@6 <= Some(20000),5,2), required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── "); Ok(()) @@ -1409,10 +1409,11 @@ mod tests { │ InterleaveExec │ ProjectionExec: expr=[store as channel, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, sum(store_sales.ss_quantity * store_sales.ss_list_price)@3 as sales, count(Int64(1))@4 as number_sales] │ NestedLoopJoinExec: join_type=Inner, filter=sum(store_sales.ss_quantity * store_sales.ss_list_price)@0 > average_sales@1, projection=[i_brand_id@1, i_class_id@2, i_category_id@3, sum(store_sales.ss_quantity * store_sales.ss_list_price)@4, count(Int64(1))@5] - │ 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 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ 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 │ 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))] @@ -1421,25 +1422,24 @@ mod tests { │ [Stage 5] => NetworkCoalesceExec: output_partitions=6, 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 - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(i_brand_id@0, brand_id@0), (i_class_id@1, class_id@1), (i_category_id@2, category_id@2)], NullsEqual: true - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=2, input_tasks=3 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=1, input_tasks=3 │ AggregateExec: mode=SinglePartitioned, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(i_brand_id@0, brand_id@0), (i_class_id@1, class_id@1), (i_category_id@2, category_id@2)], NullsEqual: true - │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=2, input_tasks=3 + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=1, input_tasks=3 │ AggregateExec: mode=FinalPartitioned, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] │ [Stage 14] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet, predicate=DynamicFilter [ empty ] │ ProjectionExec: expr=[catalog as channel, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, sum(catalog_sales.cs_quantity * catalog_sales.cs_list_price)@3 as sales, count(Int64(1))@4 as number_sales] │ NestedLoopJoinExec: join_type=Inner, filter=sum(catalog_sales.cs_quantity * catalog_sales.cs_list_price)@0 > average_sales@1, projection=[i_brand_id@1, i_class_id@2, i_category_id@3, sum(catalog_sales.cs_quantity * catalog_sales.cs_list_price)@4, count(Int64(1))@5] - │ 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 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ 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 │ 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))] @@ -1448,25 +1448,24 @@ mod tests { │ [Stage 19] => NetworkCoalesceExec: output_partitions=6, 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 - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(i_brand_id@0, brand_id@0), (i_class_id@1, class_id@1), (i_category_id@2, category_id@2)], NullsEqual: true - │ CoalescePartitionsExec - │ [Stage 22] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=2, input_tasks=3 + │ [Stage 22] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=1, input_tasks=3 │ AggregateExec: mode=SinglePartitioned, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(i_brand_id@0, brand_id@0), (i_class_id@1, class_id@1), (i_category_id@2, category_id@2)], NullsEqual: true - │ CoalescePartitionsExec - │ [Stage 25] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=2, input_tasks=3 + │ [Stage 25] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=1, input_tasks=3 │ AggregateExec: mode=FinalPartitioned, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] │ [Stage 28] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet, predicate=DynamicFilter [ empty ] │ ProjectionExec: expr=[web as channel, i_brand_id@0 as i_brand_id, i_class_id@1 as i_class_id, i_category_id@2 as i_category_id, sum(web_sales.ws_quantity * web_sales.ws_list_price)@3 as sales, count(Int64(1))@4 as number_sales] │ NestedLoopJoinExec: join_type=Inner, filter=sum(web_sales.ws_quantity * web_sales.ws_list_price)@0 > average_sales@1, projection=[i_brand_id@1, i_class_id@2, i_category_id@3, sum(web_sales.ws_quantity * web_sales.ws_list_price)@4, count(Int64(1))@5] - │ 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 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ 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 │ 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))] @@ -1475,15 +1474,13 @@ mod tests { │ [Stage 33] => NetworkCoalesceExec: output_partitions=6, 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 - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(i_brand_id@0, brand_id@0), (i_class_id@1, class_id@1), (i_category_id@2, category_id@2)], NullsEqual: true - │ CoalescePartitionsExec - │ [Stage 36] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=2, input_tasks=3 + │ [Stage 36] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=1, input_tasks=3 │ AggregateExec: mode=SinglePartitioned, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(i_brand_id@0, brand_id@0), (i_class_id@1, class_id@1), (i_category_id@2, category_id@2)], NullsEqual: true - │ CoalescePartitionsExec - │ [Stage 39] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=2, input_tasks=3 + │ [Stage 39] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=1, input_tasks=3 │ AggregateExec: mode=FinalPartitioned, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] │ [Stage 42] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 @@ -1494,8 +1491,8 @@ mod tests { │ 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 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec │ 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 @@ -1503,8 +1500,8 @@ mod tests { │ 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 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec │ 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 @@ -1512,127 +1509,129 @@ mod tests { │ 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 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 - │ 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 - │ 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 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ 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 + │ 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 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ 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 AND d_moy@2 = 11, projection=[d_date_sk@0] - │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 11, 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 <= 11 AND 11 <= d_moy_max@5, required_guarantees=[d_moy in (11), d_year in (2001)] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 AND d_moy@2 = 11, projection=[d_date_sk@0] + │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 11, 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 <= 11 AND 11 <= d_moy_max@5, required_guarantees=[d_moy in (11), d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 8 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d3.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ws_sold_date_sk@3 as ws_sold_date_sk, 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] - │ CoalescePartitionsExec - │ [Stage 7] => 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], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] - │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] - │ 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_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 8 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d3.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ ProjectionExec: expr=[ws_sold_date_sk@3 as ws_sold_date_sk, 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] + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] + │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] + │ 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_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 7 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 11 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d2.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] - │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[cs_sold_date_sk@3 as cs_sold_date_sk, 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] - │ CoalescePartitionsExec - │ [Stage 10] => 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], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] - │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] - │ 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_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 11 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d2.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ ProjectionExec: expr=[cs_sold_date_sk@3 as cs_sold_date_sk, 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] + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] + │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] + │ 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_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 10 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 14 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ RepartitionExec: partitioning=Hash([brand_id@0, class_id@1, category_id@2], 3), input_partitions=2 │ AggregateExec: mode=Partial, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] │ ProjectionExec: expr=[i_brand_id@0 as brand_id, i_class_id@1 as class_id, i_category_id@2 as category_id] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] - │ CoalescePartitionsExec - │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[ss_sold_date_sk@3 as ss_sold_date_sk, 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] - │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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], 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 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] - │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] - │ 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_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 12 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] + │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] + │ 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_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 13 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet + ┌───── Stage 13 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec │ 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 @@ -1640,8 +1639,8 @@ mod tests { │ 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 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec │ 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 @@ -1649,127 +1648,129 @@ mod tests { │ 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 + │ [Stage 15] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 - │ 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 - │ 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 15 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ 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 + │ 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 + │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ 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 AND d_moy@2 = 11, projection=[d_date_sk@0] - │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 11, 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 <= 11 AND 11 <= d_moy_max@5, required_guarantees=[d_moy in (11), d_year in (2001)] - └────────────────────────────────────────────────── - ┌───── Stage 18 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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 22 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d3.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] - │ CoalescePartitionsExec - │ [Stage 20] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ws_sold_date_sk@3 as ws_sold_date_sk, 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] - │ CoalescePartitionsExec - │ [Stage 21] => 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], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 20 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] - │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] - │ 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_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 21 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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 25 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d2.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] - │ CoalescePartitionsExec - │ [Stage 23] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[cs_sold_date_sk@3 as cs_sold_date_sk, 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] - │ CoalescePartitionsExec - │ [Stage 24] => 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], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 23 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] - │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] - │ 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_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 17 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 AND d_moy@2 = 11, projection=[d_date_sk@0] + │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 11, 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 <= 11 AND 11 <= d_moy_max@5, required_guarantees=[d_moy in (11), d_year in (2001)] + └────────────────────────────────────────────────── + ┌───── Stage 18 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 22 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d3.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] + │ [Stage 20] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ ProjectionExec: expr=[ws_sold_date_sk@3 as ws_sold_date_sk, 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] + │ [Stage 21] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 20 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] + │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] + │ 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_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 21 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 25 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d2.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] + │ [Stage 23] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ ProjectionExec: expr=[cs_sold_date_sk@3 as cs_sold_date_sk, 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] + │ [Stage 24] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 23 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] + │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] + │ 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_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 24 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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 24 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 28 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ RepartitionExec: partitioning=Hash([brand_id@0, class_id@1, category_id@2], 3), input_partitions=2 │ AggregateExec: mode=Partial, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] │ ProjectionExec: expr=[i_brand_id@0 as brand_id, i_class_id@1 as class_id, i_category_id@2 as category_id] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] - │ CoalescePartitionsExec - │ [Stage 26] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 26] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[ss_sold_date_sk@3 as ss_sold_date_sk, 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] - │ CoalescePartitionsExec - │ [Stage 27] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 27] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 26 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] - │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] - │ 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_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 26 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] + │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] + │ 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_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 27 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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 27 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec │ 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 @@ -1777,8 +1778,8 @@ mod tests { │ 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 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec │ 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 @@ -1786,119 +1787,121 @@ mod tests { │ 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 + │ [Stage 29] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 - │ 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 - │ 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 29 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ 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 + │ 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 + │ [Stage 31] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 32] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ 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 AND d_moy@2 = 11, projection=[d_date_sk@0] - │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 11, 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 <= 11 AND 11 <= d_moy_max@5, required_guarantees=[d_moy in (11), d_year in (2001)] - └────────────────────────────────────────────────── - ┌───── Stage 32 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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 36 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d3.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] - │ CoalescePartitionsExec - │ [Stage 34] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ws_sold_date_sk@3 as ws_sold_date_sk, 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] - │ CoalescePartitionsExec - │ [Stage 35] => 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], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 34 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] - │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] - │ 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_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 35 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 39 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d2.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] - │ CoalescePartitionsExec - │ [Stage 37] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[cs_sold_date_sk@3 as cs_sold_date_sk, 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] - │ CoalescePartitionsExec - │ [Stage 38] => 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], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 37 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] - │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] - │ 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_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 31 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 AND d_moy@2 = 11, projection=[d_date_sk@0] + │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 11, 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 <= 11 AND 11 <= d_moy_max@5, required_guarantees=[d_moy in (11), d_year in (2001)] + └────────────────────────────────────────────────── + ┌───── Stage 32 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 36 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d3.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] + │ [Stage 34] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ ProjectionExec: expr=[ws_sold_date_sk@3 as ws_sold_date_sk, 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] + │ [Stage 35] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 34 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] + │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] + │ 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_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 35 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 39 ── Tasks: t0:[p0] t1:[p1] t2:[p2] + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d2.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] + │ [Stage 37] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ ProjectionExec: expr=[cs_sold_date_sk@3 as cs_sold_date_sk, 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] + │ [Stage 38] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 37 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] + │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] + │ 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_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 38 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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 38 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand_id, i_class_id, i_category_id], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 42 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ RepartitionExec: partitioning=Hash([brand_id@0, class_id@1, category_id@2], 3), input_partitions=2 │ AggregateExec: mode=Partial, gby=[brand_id@0 as brand_id, class_id@1 as class_id, category_id@2 as category_id], aggr=[] │ ProjectionExec: expr=[i_brand_id@0 as brand_id, i_class_id@1 as class_id, i_category_id@2 as category_id] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[i_brand_id@3, i_class_id@4, i_category_id@5] - │ CoalescePartitionsExec - │ [Stage 40] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 40] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[ss_sold_date_sk@3 as ss_sold_date_sk, 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] - │ CoalescePartitionsExec - │ [Stage 41] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 41] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 40 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] - │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] - │ 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_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 40 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] + │ FilterExec: d_year@1 >= 1999 AND d_year@1 <= 2001, projection=[d_date_sk@0] + │ 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_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 41 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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 41 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 └────────────────────────────────────────────────── "); Ok(()) @@ -1920,29 +1923,29 @@ mod tests { │ RepartitionExec: partitioning=Hash([ca_zip@0], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[ca_zip@1 as ca_zip], aggr=[sum(catalog_sales.cs_sales_price)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_sales_price@3, ca_zip@4] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_sales_price@2 as cs_sales_price, ca_zip@0 as ca_zip] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@2)], filter=substr(ca_zip@2, 1, 5) IN (SET) ([85669, 86197, 88274, 83405, 86475, 85392, 85460, 80348, 81792]) OR ca_state@1 = CA OR ca_state@1 = WA OR ca_state@1 = GA OR cs_sales_price@0 > Some(50000),5,2, projection=[ca_zip@2, cs_sold_date_sk@3, cs_sales_price@4] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_sales_price@2 as cs_sales_price, c_current_addr_sk@0 as c_current_addr_sk] │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@2, cs_bill_customer_sk@1)], projection=[c_current_addr_sk@1, cs_sold_date_sk@3, cs_sales_price@5] │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_qoy@2 = 2 AND d_year@1 = 2001, projection=[d_date_sk@0] - │ 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_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 2 AND d_year@6 = 2001, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 2 AND 2 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_qoy in (2), d_year in (2001)] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_qoy@2 = 2 AND d_year@1 = 2001, projection=[d_date_sk@0] + │ 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_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 2 AND d_year@6 = 2001, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 2 AND 2 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_qoy in (2), d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state, ca_zip], file_type=parquet + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state, ca_zip], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p0..p8] │ RepartitionExec: partitioning=Hash([CAST(customer.c_customer_sk AS Float64)@2], 9), input_partitions=3 @@ -1982,40 +1985,40 @@ mod tests { └────────────────────────────────────────────────── ┌───── 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 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ ProjectionExec: expr=[cc_call_center_sk@0 as cc_call_center_sk, CAST(cc_call_center_sk@0 AS Float64) as CAST(call_center.cc_call_center_sk AS Float64)] - │ FilterExec: cc_county@1 = Williamson County, projection=[cc_call_center_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/call_center/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-3.parquet]]}, projection=[cc_call_center_sk, cc_county], file_type=parquet, predicate=cc_county@25 = Williamson County, pruning_predicate=cc_county_null_count@2 != row_count@3 AND cc_county_min@0 <= Williamson County AND Williamson County <= cc_county_max@1, required_guarantees=[cc_county in (Williamson County)] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[cc_call_center_sk@0 as cc_call_center_sk, CAST(cc_call_center_sk@0 AS Float64) as CAST(call_center.cc_call_center_sk AS Float64)] + │ FilterExec: cc_county@1 = Williamson County, projection=[cc_call_center_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/call_center/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-3.parquet]]}, projection=[cc_call_center_sk, cc_county], file_type=parquet, predicate=cc_county@25 = Williamson County, pruning_predicate=cc_county_null_count@2 != row_count@3 AND cc_county_min@0 <= Williamson County AND Williamson County <= cc_county_max@1, required_guarantees=[cc_county in (Williamson County)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ FilterExec: ca_state@1 = GA, projection=[ca_address_sk@0] - │ 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_state], file_type=parquet, predicate=ca_state@8 = GA, pruning_predicate=ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= GA AND GA <= ca_state_max@1, required_guarantees=[ca_state in (GA)] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] + │ FilterExec: ca_state@1 = GA, projection=[ca_address_sk@0] + │ 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_state], file_type=parquet, predicate=ca_state@8 = GA, pruning_predicate=ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= GA AND GA <= ca_state_max@1, required_guarantees=[ca_state in (GA)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_date@1 >= 2002-02-01 AND d_date@1 <= 2002-04-02, projection=[d_date_sk@0] - │ 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, predicate=d_date@2 >= 2002-02-01 AND d_date@2 <= 2002-04-02, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2002-02-01 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2002-04-02, required_guarantees=[] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_date@1 >= 2002-02-01 AND d_date@1 <= 2002-04-02, projection=[d_date_sk@0] + │ 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, predicate=d_date@2 >= 2002-02-01 AND d_date@2 <= 2002-04-02, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2002-02-01 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2002-04-02, required_guarantees=[] └────────────────────────────────────────────────── "); Ok(()) @@ -2038,60 +2041,60 @@ mod tests { │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, s_state@2], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[i_item_id@4 as i_item_id, i_item_desc@5 as i_item_desc, s_state@3 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)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_quantity@1, sr_return_quantity@2, cs_quantity@3, s_state@4, i_item_id@6, i_item_desc@7] - │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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, i_item_desc], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_quantity@2 as ss_quantity, sr_return_quantity@3 as sr_return_quantity, cs_quantity@4 as cs_quantity, s_state@0 as s_state] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, ss_store_sk@1)], projection=[s_state@1, ss_item_sk@3, ss_quantity@5, sr_return_quantity@6, cs_quantity@7] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d3.d_date_sk AS Float64)@1, cs_sold_date_sk@4)], projection=[ss_item_sk@2, ss_store_sk@3, ss_quantity@4, sr_return_quantity@5, cs_quantity@7] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d2.d_date_sk AS Float64)@1, sr_returned_date_sk@3)], projection=[ss_item_sk@2, ss_store_sk@3, ss_quantity@4, sr_return_quantity@6, cs_sold_date_sk@7, cs_quantity@8] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_quantity@5, sr_returned_date_sk@6, sr_return_quantity@7, cs_sold_date_sk@8, cs_quantity@9] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ss_sold_date_sk@2 as ss_sold_date_sk, ss_item_sk@3 as ss_item_sk, ss_store_sk@4 as ss_store_sk, ss_quantity@5 as ss_quantity, sr_returned_date_sk@6 as sr_returned_date_sk, sr_return_quantity@7 as sr_return_quantity, cs_sold_date_sk@0 as cs_sold_date_sk, cs_quantity@1 as cs_quantity] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(cs_bill_customer_sk@1, sr_customer_sk@6), (cs_item_sk@2, sr_item_sk@5)], projection=[cs_sold_date_sk@0, cs_quantity@3, ss_sold_date_sk@4, ss_item_sk@5, ss_store_sk@6, ss_quantity@7, sr_returned_date_sk@8, sr_return_quantity@11] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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_state, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] - │ FilterExec: d_quarter_name@1 = 2001Q1 OR d_quarter_name@1 = 2001Q2 OR d_quarter_name@1 = 2001Q3, projection=[d_date_sk@0] - │ 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_quarter_name], file_type=parquet, predicate=d_quarter_name@15 = 2001Q1 OR d_quarter_name@15 = 2001Q2 OR d_quarter_name@15 = 2001Q3, pruning_predicate=d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q1 AND 2001Q1 <= d_quarter_name_max@1 OR d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q2 AND 2001Q2 <= d_quarter_name_max@1 OR d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q3 AND 2001Q3 <= d_quarter_name_max@1, required_guarantees=[d_quarter_name in (2001Q1, 2001Q2, 2001Q3)] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] - │ FilterExec: d_quarter_name@1 = 2001Q1 OR d_quarter_name@1 = 2001Q2 OR d_quarter_name@1 = 2001Q3, projection=[d_date_sk@0] - │ 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_quarter_name], file_type=parquet, predicate=d_quarter_name@15 = 2001Q1 OR d_quarter_name@15 = 2001Q2 OR d_quarter_name@15 = 2001Q3, pruning_predicate=d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q1 AND 2001Q1 <= d_quarter_name_max@1 OR d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q2 AND 2001Q2 <= d_quarter_name_max@1 OR d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q3 AND 2001Q3 <= d_quarter_name_max@1, required_guarantees=[d_quarter_name in (2001Q1, 2001Q2, 2001Q3)] + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_quantity@2 as ss_quantity, sr_return_quantity@3 as sr_return_quantity, cs_quantity@4 as cs_quantity, s_state@0 as s_state] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, ss_store_sk@1)], projection=[s_state@1, ss_item_sk@3, ss_quantity@5, sr_return_quantity@6, cs_quantity@7] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d3.d_date_sk AS Float64)@1, cs_sold_date_sk@4)], projection=[ss_item_sk@2, ss_store_sk@3, ss_quantity@4, sr_return_quantity@5, cs_quantity@7] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d2.d_date_sk AS Float64)@1, sr_returned_date_sk@3)], projection=[ss_item_sk@2, ss_store_sk@3, ss_quantity@4, sr_return_quantity@6, cs_sold_date_sk@7, cs_quantity@8] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_quantity@5, sr_returned_date_sk@6, sr_return_quantity@7, cs_sold_date_sk@8, cs_quantity@9] + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ ProjectionExec: expr=[ss_sold_date_sk@2 as ss_sold_date_sk, ss_item_sk@3 as ss_item_sk, ss_store_sk@4 as ss_store_sk, ss_quantity@5 as ss_quantity, sr_returned_date_sk@6 as sr_returned_date_sk, sr_return_quantity@7 as sr_return_quantity, cs_sold_date_sk@0 as cs_sold_date_sk, cs_quantity@1 as cs_quantity] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(cs_bill_customer_sk@1, sr_customer_sk@6), (cs_item_sk@2, sr_item_sk@5)], projection=[cs_sold_date_sk@0, cs_quantity@3, ss_sold_date_sk@4, ss_item_sk@5, ss_store_sk@6, ss_quantity@7, sr_returned_date_sk@8, sr_return_quantity@11] + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_state, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] + │ FilterExec: d_quarter_name@1 = 2001Q1 OR d_quarter_name@1 = 2001Q2 OR d_quarter_name@1 = 2001Q3, projection=[d_date_sk@0] + │ 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_quarter_name], file_type=parquet, predicate=d_quarter_name@15 = 2001Q1 OR d_quarter_name@15 = 2001Q2 OR d_quarter_name@15 = 2001Q3, pruning_predicate=d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q1 AND 2001Q1 <= d_quarter_name_max@1 OR d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q2 AND 2001Q2 <= d_quarter_name_max@1 OR d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q3 AND 2001Q3 <= d_quarter_name_max@1, required_guarantees=[d_quarter_name in (2001Q1, 2001Q2, 2001Q3)] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] - │ FilterExec: d_quarter_name@1 = 2001Q1, projection=[d_date_sk@0] - │ 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_quarter_name], file_type=parquet, predicate=d_quarter_name@15 = 2001Q1, pruning_predicate=d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q1 AND 2001Q1 <= d_quarter_name_max@1, required_guarantees=[d_quarter_name in (2001Q1)] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] + │ FilterExec: d_quarter_name@1 = 2001Q1 OR d_quarter_name@1 = 2001Q2 OR d_quarter_name@1 = 2001Q3, projection=[d_date_sk@0] + │ 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_quarter_name], file_type=parquet, predicate=d_quarter_name@15 = 2001Q1 OR d_quarter_name@15 = 2001Q2 OR d_quarter_name@15 = 2001Q3, pruning_predicate=d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q1 AND 2001Q1 <= d_quarter_name_max@1 OR d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q2 AND 2001Q2 <= d_quarter_name_max@1 OR d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q3 AND 2001Q3 <= d_quarter_name_max@1, required_guarantees=[d_quarter_name in (2001Q1, 2001Q2, 2001Q3)] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] + │ FilterExec: d_quarter_name@1 = 2001Q1, projection=[d_date_sk@0] + │ 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_quarter_name], file_type=parquet, predicate=d_quarter_name@15 = 2001Q1, pruning_predicate=d_quarter_name_null_count@2 != row_count@3 AND d_quarter_name_min@0 <= 2001Q1 AND 2001Q1 <= d_quarter_name_max@1, required_guarantees=[d_quarter_name in (2001Q1)] └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@1, cs_item_sk@2], 9), input_partitions=2 @@ -2136,58 +2139,58 @@ mod tests { │ RepartitionExec: partitioning=Hash([i_item_id@0, ca_country@1, ca_state@2, ca_county@3, __grouping_id@4], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[(NULL as i_item_id, NULL as ca_country, NULL as ca_state, NULL as ca_county), (i_item_id@10 as i_item_id, NULL as ca_country, NULL as ca_state, NULL as ca_county), (i_item_id@10 as i_item_id, ca_country@9 as ca_country, NULL as ca_state, NULL as ca_county), (i_item_id@10 as i_item_id, ca_country@9 as ca_country, ca_state@8 as ca_state, NULL as ca_county), (i_item_id@10 as i_item_id, ca_country@9 as ca_country, ca_state@8 as ca_state, ca_county@7 as ca_county)], 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)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, i_item_sk@0)], projection=[cs_quantity@1, cs_list_price@2, cs_sales_price@3, cs_coupon_amt@4, cs_net_profit@5, cd_dep_count@6, c_birth_year@7, ca_county@8, ca_state@9, ca_country@10, i_item_id@12] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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, cs_sales_price@6, cs_coupon_amt@7, cs_net_profit@8, cd_dep_count@9, c_birth_year@10, ca_county@11, ca_state@12, ca_country@13] - │ CoalescePartitionsExec - │ [Stage 1] => 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, cs_sales_price@7 as cs_sales_price, cs_coupon_amt@8 as cs_coupon_amt, cs_net_profit@9 as cs_net_profit, cd_dep_count@10 as cd_dep_count, c_birth_year@11 as c_birth_year, ca_county@0 as ca_county, ca_state@1 as ca_state, ca_country@2 as ca_country] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@8)], projection=[ca_county@1, ca_state@2, ca_country@3, cs_sold_date_sk@4, cs_item_sk@5, cs_quantity@6, cs_list_price@7, cs_sales_price@8, cs_coupon_amt@9, cs_net_profit@10, cd_dep_count@11, c_birth_year@13] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_current_cdemo_sk@8, CAST(cd2.cd_demo_sk AS Float64)@1)], projection=[cs_sold_date_sk@0, cs_item_sk@1, cs_quantity@2, cs_list_price@3, cs_sales_price@4, cs_coupon_amt@5, cs_net_profit@6, cd_dep_count@7, c_current_addr_sk@9, c_birth_year@10] - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 = 1998, projection=[d_date_sk@0] + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ 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, cs_sales_price@6, cs_coupon_amt@7, cs_net_profit@8, cd_dep_count@9, c_birth_year@10, ca_county@11, ca_state@12, ca_country@13] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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, cs_sales_price@7 as cs_sales_price, cs_coupon_amt@8 as cs_coupon_amt, cs_net_profit@9 as cs_net_profit, cd_dep_count@10 as cd_dep_count, c_birth_year@11 as c_birth_year, ca_county@0 as ca_county, ca_state@1 as ca_state, ca_country@2 as ca_country] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@8)], projection=[ca_county@1, ca_state@2, ca_country@3, cs_sold_date_sk@4, cs_item_sk@5, cs_quantity@6, cs_list_price@7, cs_sales_price@8, cs_coupon_amt@9, cs_net_profit@10, cd_dep_count@11, c_birth_year@13] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_current_cdemo_sk@8, CAST(cd2.cd_demo_sk AS Float64)@1)], projection=[cs_sold_date_sk@0, cs_item_sk@1, cs_quantity@2, cs_list_price@3, cs_sales_price@4, cs_coupon_amt@5, cs_net_profit@6, cd_dep_count@7, c_current_addr_sk@9, c_birth_year@10] + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 = 1998, projection=[d_date_sk@0] + │ 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_year], file_type=parquet, predicate=d_year@6 = 1998, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1, required_guarantees=[d_year in (1998)] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: ca_state@2 IN (SET) ([MS, IN, ND, OK, NM, VA, MS]) │ 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_year], file_type=parquet, predicate=d_year@6 = 1998, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1998 AND 1998 <= d_year_max@1, required_guarantees=[d_year in (1998)] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: ca_state@2 IN (SET) ([MS, IN, ND, OK, NM, VA, MS]) - │ 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_county, ca_state, ca_country], file_type=parquet, predicate=ca_state@8 IN (SET) ([MS, IN, ND, OK, NM, VA, MS]), pruning_predicate=ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= MS AND MS <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IN AND IN <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= ND AND ND <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OK AND OK <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= NM AND NM <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= VA AND VA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= MS AND MS <= ca_state_max@1, required_guarantees=[ca_state in (IN, MS, ND, NM, OK, VA)] + │ 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_county, ca_state, ca_country], file_type=parquet, predicate=ca_state@8 IN (SET) ([MS, IN, ND, OK, NM, VA, MS]), pruning_predicate=ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= MS AND MS <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IN AND IN <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= ND AND ND <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OK AND OK <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= NM AND NM <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= VA AND VA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= MS AND MS <= ca_state_max@1, required_guarantees=[ca_state in (IN, MS, ND, NM, OK, VA)] └────────────────────────────────────────────────── ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ RepartitionExec: partitioning=Hash([c_current_cdemo_sk@8], 9), input_partitions=3 │ 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, cs_sales_price@7 as cs_sales_price, cs_coupon_amt@8 as cs_coupon_amt, cs_net_profit@9 as cs_net_profit, cd_dep_count@10 as cd_dep_count, c_current_cdemo_sk@0 as c_current_cdemo_sk, c_current_addr_sk@1 as c_current_addr_sk, c_birth_year@2 as c_birth_year] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@4, cs_bill_customer_sk@1)], projection=[c_current_cdemo_sk@1, c_current_addr_sk@2, c_birth_year@3, cs_sold_date_sk@5, cs_item_sk@7, cs_quantity@8, cs_list_price@9, cs_sales_price@10, cs_coupon_amt@11, cs_net_profit@12, cd_dep_count@13] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_bill_customer_sk@2 as cs_bill_customer_sk, cs_item_sk@3 as cs_item_sk, cs_quantity@4 as cs_quantity, cs_list_price@5 as cs_list_price, cs_sales_price@6 as cs_sales_price, cs_coupon_amt@7 as cs_coupon_amt, cs_net_profit@8 as cs_net_profit, cd_dep_count@0 as cd_dep_count] │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(cd1.cd_demo_sk AS Float64)@2, cs_bill_cdemo_sk@2)], projection=[cd_dep_count@1, cs_sold_date_sk@3, cs_bill_customer_sk@4, cs_item_sk@6, cs_quantity@7, cs_list_price@8, cs_sales_price@9, cs_coupon_amt@10, cs_net_profit@11] │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_cdemo_sk@1 as c_current_cdemo_sk, c_current_addr_sk@2 as c_current_addr_sk, c_birth_year@3 as c_birth_year, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ FilterExec: c_birth_month@3 IN (SET) ([1, 6, 8, 9, 12, 2]), projection=[c_customer_sk@0, c_current_cdemo_sk@1, c_current_addr_sk@2, c_birth_year@4] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_addr_sk, c_birth_month, c_birth_year], file_type=parquet, predicate=c_birth_month@12 IN (SET) ([1, 6, 8, 9, 12, 2]) AND DynamicFilter [ empty ], pruning_predicate=c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 1 AND 1 <= c_birth_month_max@1 OR c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 6 AND 6 <= c_birth_month_max@1 OR c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 8 AND 8 <= c_birth_month_max@1 OR c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 9 AND 9 <= c_birth_month_max@1 OR c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 12 AND 12 <= c_birth_month_max@1 OR c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 2 AND 2 <= c_birth_month_max@1, required_guarantees=[c_birth_month in (1, 12, 2, 6, 8, 9)] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_cdemo_sk@1 as c_current_cdemo_sk, c_current_addr_sk@2 as c_current_addr_sk, c_birth_year@3 as c_birth_year, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] + │ FilterExec: c_birth_month@3 IN (SET) ([1, 6, 8, 9, 12, 2]), projection=[c_customer_sk@0, c_current_cdemo_sk@1, c_current_addr_sk@2, c_birth_year@4] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_addr_sk, c_birth_month, c_birth_year], file_type=parquet, predicate=c_birth_month@12 IN (SET) ([1, 6, 8, 9, 12, 2]) AND DynamicFilter [ empty ], pruning_predicate=c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 1 AND 1 <= c_birth_month_max@1 OR c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 6 AND 6 <= c_birth_month_max@1 OR c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 8 AND 8 <= c_birth_month_max@1 OR c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 9 AND 9 <= c_birth_month_max@1 OR c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 12 AND 12 <= c_birth_month_max@1 OR c_birth_month_null_count@2 != row_count@3 AND c_birth_month_min@0 <= 2 AND 2 <= c_birth_month_max@1, required_guarantees=[c_birth_month in (1, 12, 2, 6, 8, 9)] └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ RepartitionExec: partitioning=Hash([CAST(cd1.cd_demo_sk AS Float64)@2], 9), input_partitions=2 @@ -2227,51 +2230,51 @@ mod tests { │ RepartitionExec: partitioning=Hash([i_brand@0, i_brand_id@1, i_manufact_id@2, i_manufact@3], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[i_brand@2 as i_brand, i_brand_id@1 as i_brand_id, i_manufact_id@3 as i_manufact_id, i_manufact@4 as i_manufact], aggr=[sum(store_sales.ss_ext_sales_price)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, ss_store_sk@0)], filter=substr(ca_zip@0, 1, 5) != substr(s_zip@1, 1, 5), projection=[ss_ext_sales_price@4, i_brand_id@5, i_brand@6, i_manufact_id@7, i_manufact@8] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[ss_store_sk@1 as ss_store_sk, ss_ext_sales_price@2 as ss_ext_sales_price, i_brand_id@3 as i_brand_id, i_brand@4 as i_brand, i_manufact_id@5 as i_manufact_id, i_manufact@6 as i_manufact, ca_zip@0 as ca_zip] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@6)], projection=[ca_zip@1, ss_store_sk@2, ss_ext_sales_price@3, i_brand_id@4, i_brand@5, i_manufact_id@6, i_manufact@7] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ss_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@2)], projection=[ss_store_sk@1, ss_ext_sales_price@2, i_brand_id@3, i_brand@4, i_manufact_id@5, i_manufact@6, c_current_addr_sk@8] │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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_zip], file_type=parquet + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_zip], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ RepartitionExec: partitioning=Hash([ss_customer_sk@0], 9), input_partitions=3 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_customer_sk@1, ss_store_sk@2, ss_ext_sales_price@3, i_brand_id@5, i_brand@6, i_manufact_id@7, i_manufact@8] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ FilterExec: i_manager_id@5 = 8, projection=[i_item_sk@0, i_brand_id@1, i_brand@2, i_manufact_id@3, i_manufact@4] │ 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_brand_id, i_brand, i_manufact_id, i_manufact, i_manager_id], file_type=parquet, predicate=i_manager_id@20 = 8 AND DynamicFilter [ empty ], pruning_predicate=i_manager_id_null_count@2 != row_count@3 AND i_manager_id_min@0 <= 8 AND 8 <= i_manager_id_max@1, required_guarantees=[i_manager_id in (8)] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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_customer_sk@4, ss_store_sk@5, ss_ext_sales_price@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/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_store_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ 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_customer_sk@4, ss_store_sk@5, ss_ext_sales_price@6] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_customer_sk, ss_store_sk, ss_ext_sales_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 - │ 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_moy@2 = 11 AND d_year@1 = 1998, projection=[d_date_sk@0] - │ 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_year, d_moy], file_type=parquet, predicate=d_moy@8 = 11 AND d_year@6 = 1998, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1998 AND 1998 <= d_year_max@5, required_guarantees=[d_moy in (11), d_year in (1998)] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_moy@2 = 11 AND d_year@1 = 1998, projection=[d_date_sk@0] + │ 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_year, d_moy], file_type=parquet, predicate=d_moy@8 = 11 AND d_year@6 = 1998, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1998 AND 1998 <= d_year_max@5, required_guarantees=[d_moy in (11), d_year in (1998)] └────────────────────────────────────────────────── ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p0..p8] │ RepartitionExec: partitioning=Hash([CAST(customer.c_customer_sk AS Float64)@2], 9), input_partitions=3 @@ -2304,29 +2307,29 @@ mod tests { │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, i_category@2, i_class@3, i_current_price@4], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id, i_item_desc@2 as i_item_desc, i_category@5 as i_category, i_class@4 as i_class, i_current_price@3 as i_current_price], aggr=[sum(catalog_sales.cs_ext_sales_price)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_ext_sales_price@3, i_item_id@4, i_item_desc@5, i_current_price@6, i_class@7, i_category@8] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[cs_sold_date_sk@5 as cs_sold_date_sk, cs_ext_sales_price@6 as cs_ext_sales_price, i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_current_price@2 as i_current_price, i_class@3 as i_class, i_category@4 as i_category] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_item_id@1, i_item_desc@2, i_current_price@3, i_class@4, i_category@5, cs_sold_date_sk@6, cs_ext_sales_price@8] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_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 - │ 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_date@1 >= 1999-02-22 AND d_date@1 <= 1999-03-24, projection=[d_date_sk@0] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_date@1 >= 1999-02-22 AND d_date@1 <= 1999-03-24, projection=[d_date_sk@0] + │ 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, predicate=d_date@2 >= 1999-02-22 AND d_date@2 <= 1999-03-24, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 1999-02-22 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 1999-03-24, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: i_category@5 = Sports OR i_category@5 = Books OR i_category@5 = Home │ 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, predicate=d_date@2 >= 1999-02-22 AND d_date@2 <= 1999-03-24, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 1999-02-22 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 1999-03-24, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_category@5 = Sports OR i_category@5 = Books OR i_category@5 = Home - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id, i_item_desc, i_current_price, i_class, i_category], file_type=parquet, predicate=i_category@12 = Sports OR i_category@12 = Books OR i_category@12 = Home, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Sports AND Sports <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Home AND Home <= i_category_max@1, required_guarantees=[i_category in (Books, Home, Sports)] + │ 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, i_item_desc, i_current_price, i_class, i_category], file_type=parquet, predicate=i_category@12 = Sports OR i_category@12 = Books OR i_category@12 = Home, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Sports AND Sports <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Home AND Home <= i_category_max@1, required_guarantees=[i_category in (Books, Home, Sports)] └────────────────────────────────────────────────── "#); Ok(()) @@ -2352,37 +2355,37 @@ mod tests { │ AggregateExec: mode=Partial, gby=[w_warehouse_name@2 as w_warehouse_name, i_item_id@3 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)] │ ProjectionExec: expr=[d_date@0 as __common_expr_2, inv_quantity_on_hand@1 as inv_quantity_on_hand, w_warehouse_name@2 as w_warehouse_name, i_item_id@3 as i_item_id] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_date_sk@0, inv_date_sk@0)], projection=[d_date@1, inv_quantity_on_hand@3, w_warehouse_name@4, i_item_id@5] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[inv_date_sk@1 as inv_date_sk, inv_quantity_on_hand@2 as inv_quantity_on_hand, w_warehouse_name@3 as w_warehouse_name, i_item_id@0 as i_item_id] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, inv_item_sk@1)], projection=[i_item_id@1, inv_date_sk@2, inv_quantity_on_hand@4, w_warehouse_name@5] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[inv_date_sk@1 as inv_date_sk, inv_item_sk@2 as inv_item_sk, inv_quantity_on_hand@3 as inv_quantity_on_hand, w_warehouse_name@0 as w_warehouse_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(w_warehouse_sk@0, inv_warehouse_sk@2)], projection=[w_warehouse_name@1, inv_date_sk@2, inv_item_sk@3, inv_quantity_on_hand@5] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/inventory/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], ...]}, projection=[inv_date_sk, inv_item_sk, inv_warehouse_sk, inv_quantity_on_hand], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: d_date@1 >= 2000-02-10 AND d_date@1 <= 2000-04-10 - │ 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, predicate=d_date@2 >= 2000-02-10 AND d_date@2 <= 2000-04-10, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-02-10 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-10, required_guarantees=[] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: d_date@1 >= 2000-02-10 AND d_date@1 <= 2000-04-10 + │ 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, predicate=d_date@2 >= 2000-02-10 AND d_date@2 <= 2000-04-10, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-02-10 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-10, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_current_price@2 >= Some(99),4,2 AND i_current_price@2 <= Some(149),4,2, projection=[i_item_sk@0, i_item_id@1] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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, i_current_price], file_type=parquet, predicate=i_current_price@5 >= Some(99),4,2 AND i_current_price@5 <= Some(149),4,2, pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 >= Some(99),4,2 AND i_current_price_null_count@1 != row_count@2 AND i_current_price_min@3 <= Some(149),4,2, required_guarantees=[] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: i_current_price@2 >= Some(99),4,2 AND i_current_price@2 <= Some(149),4,2, projection=[i_item_sk@0, i_item_id@1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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, i_current_price], file_type=parquet, predicate=i_current_price@5 >= Some(99),4,2 AND i_current_price@5 <= Some(149),4,2, pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 >= Some(99),4,2 AND i_current_price_null_count@1 != row_count@2 AND i_current_price_min@3 <= Some(149),4,2, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_warehouse_name], file_type=parquet + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_warehouse_name], file_type=parquet └────────────────────────────────────────────────── "#); Ok(()) @@ -2406,25 +2409,25 @@ mod tests { │ AggregateExec: mode=Partial, gby=[(NULL as i_product_name, NULL as i_brand, NULL as i_class, NULL as i_category), (i_product_name@4 as i_product_name, NULL as i_brand, NULL as i_class, NULL as i_category), (i_product_name@4 as i_product_name, i_brand@1 as i_brand, NULL as i_class, NULL as i_category), (i_product_name@4 as i_product_name, i_brand@1 as i_brand, i_class@2 as i_class, NULL as i_category), (i_product_name@4 as i_product_name, i_brand@1 as i_brand, i_class@2 as i_class, i_category@3 as i_category)], aggr=[avg(inventory.inv_quantity_on_hand)] │ ProjectionExec: expr=[inv_quantity_on_hand@4 as inv_quantity_on_hand, i_brand@0 as i_brand, i_class@1 as i_class, i_category@2 as i_category, i_product_name@3 as i_product_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, inv_item_sk@0)], projection=[i_brand@1, i_class@2, i_category@3, i_product_name@4, inv_quantity_on_hand@6] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_date_sk@0, inv_date_sk@0)], projection=[inv_item_sk@2, inv_quantity_on_hand@3] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/inventory/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], ...]}, projection=[inv_date_sk, inv_item_sk, inv_quantity_on_hand], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_class, i_category, i_product_name], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_class, i_category, i_product_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: d_month_seq@1 >= 1200 AND d_month_seq@1 <= 1211, projection=[d_date_sk@0] - │ 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_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: d_month_seq@1 >= 1200 AND d_month_seq@1 <= 1211, projection=[d_date_sk@0] + │ 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_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── "); Ok(()) @@ -2457,68 +2460,68 @@ mod tests { ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@0], 9), input_partitions=3 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@1, item_sk@0)], projection=[cs_bill_customer_sk@0, cs_quantity@2, cs_list_price@3, c_first_name@4, c_last_name@5] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[i_item_sk@0 as item_sk] │ FilterExec: count(Int64(1))@1 > 4, projection=[i_item_sk@0] │ ProjectionExec: expr=[i_item_sk@1 as i_item_sk, count(Int64(1))@3 as count(Int64(1))] │ AggregateExec: mode=FinalPartitioned, gby=[itemdesc@0 as itemdesc, i_item_sk@1 as i_item_sk, d_date@2 as d_date], aggr=[count(Int64(1))] │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_bill_customer_sk@3, cs_item_sk@4, cs_quantity@5, cs_list_price@6, c_first_name@7, c_last_name@8] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[cs_sold_date_sk@2 as cs_sold_date_sk, cs_bill_customer_sk@3 as cs_bill_customer_sk, cs_item_sk@4 as cs_item_sk, cs_quantity@5 as cs_quantity, cs_list_price@6 as cs_list_price, c_first_name@0 as c_first_name, c_last_name@1 as c_last_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@3, cs_bill_customer_sk@1)], projection=[c_first_name@1, c_last_name@2, cs_sold_date_sk@4, cs_bill_customer_sk@5, cs_item_sk@6, cs_quantity@7, cs_list_price@8] - │ 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_bill_customer_sk, cs_item_sk, cs_quantity, cs_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_bill_customer_sk@3, cs_item_sk@4, cs_quantity@5, cs_list_price@6, c_first_name@7, c_last_name@8] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ ProjectionExec: expr=[cs_sold_date_sk@2 as cs_sold_date_sk, cs_bill_customer_sk@3 as cs_bill_customer_sk, cs_item_sk@4 as cs_item_sk, cs_quantity@5 as cs_quantity, cs_list_price@6 as cs_list_price, c_first_name@0 as c_first_name, c_last_name@1 as c_last_name] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@3, cs_bill_customer_sk@1)], projection=[c_first_name@1, c_last_name@2, cs_sold_date_sk@4, cs_bill_customer_sk@5, cs_item_sk@6, cs_quantity@7, cs_list_price@8] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_customer_sk, cs_item_sk, cs_quantity, cs_list_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 - │ 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 AND d_moy@2 = 2, projection=[d_date_sk@0] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 AND d_moy@2 = 2, projection=[d_date_sk@0] + │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 2000 AND d_moy@8 = 2, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= 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 (2000)] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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/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 = 2000 AND d_moy@8 = 2, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= 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 (2000)] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 + │ 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..p8] t1:[p0..p8] t2:[p0..p8] │ RepartitionExec: partitioning=Hash([itemdesc@0, i_item_sk@1, d_date@2], 9), input_partitions=3 │ AggregateExec: mode=Partial, gby=[itemdesc@1 as itemdesc, i_item_sk@2 as i_item_sk, d_date@0 as d_date], aggr=[count(Int64(1))] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@1)], projection=[d_date@1, itemdesc@2, i_item_sk@3] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[substr(i_item_desc@1, 1, 30) as itemdesc, i_item_sk@0 as i_item_sk] │ 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_desc], 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 - │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, d_date@0 as d_date] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_date@1, ss_item_sk@4] - │ CoalescePartitionsExec - │ [Stage 4] => 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], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, d_date@0 as d_date] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_date@1, ss_item_sk@4] + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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)] - │ FilterExec: d_year@2 IN (SET) ([2000, 2001, 2002, 2003]), projection=[d_date_sk@0, d_date@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_date_sk, d_date, d_year], file_type=parquet, predicate=d_year@6 IN (SET) ([2000, 2001, 2002, 2003]), pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2003 AND 2003 <= d_year_max@1, required_guarantees=[d_year in (2000, 2001, 2002, 2003)] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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)] + │ FilterExec: d_year@2 IN (SET) ([2000, 2001, 2002, 2003]), projection=[d_date_sk@0, d_date@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_date_sk, d_date, d_year], file_type=parquet, predicate=d_year@6 IN (SET) ([2000, 2001, 2002, 2003]), pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2003 AND 2003 <= d_year_max@1, required_guarantees=[d_year in (2000, 2001, 2002, 2003)] └────────────────────────────────────────────────── ┌───── Stage 15 ── Tasks: t0:[p0..p8] │ RepartitionExec: partitioning=Hash([CAST(best_ss_customer.c_customer_sk AS Float64)@1], 9), input_partitions=3 @@ -2548,20 +2551,20 @@ mod tests { │ RepartitionExec: partitioning=Hash([c_customer_sk@0], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[c_customer_sk@2 as c_customer_sk], aggr=[sum(store_sales.ss_quantity * store_sales.ss_sales_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_sales_price@4, c_customer_sk@5] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[ss_sold_date_sk@1 as ss_sold_date_sk, ss_quantity@2 as ss_quantity, ss_sales_price@3 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@1)], projection=[c_customer_sk@0, ss_sold_date_sk@2, ss_quantity@4, ss_sales_price@5] │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=2 │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 IN (SET) ([2000, 2001, 2002, 2003]), projection=[d_date_sk@0] - │ 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_year], file_type=parquet, predicate=d_year@6 IN (SET) ([2000, 2001, 2002, 2003]), pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2003 AND 2003 <= d_year_max@1, required_guarantees=[d_year in (2000, 2001, 2002, 2003)] + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 IN (SET) ([2000, 2001, 2002, 2003]), projection=[d_date_sk@0] + │ 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_year], file_type=parquet, predicate=d_year@6 IN (SET) ([2000, 2001, 2002, 2003]), pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2003 AND 2003 <= d_year_max@1, required_guarantees=[d_year in (2000, 2001, 2002, 2003)] └────────────────────────────────────────────────── ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p0..p8] │ RepartitionExec: partitioning=Hash([CAST(customer.c_customer_sk AS Float64)@1], 9), input_partitions=3 @@ -2597,68 +2600,68 @@ mod tests { ┌───── Stage 23 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ RepartitionExec: partitioning=Hash([ws_bill_customer_sk@0], 9), input_partitions=3 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_item_sk@0, item_sk@0)], projection=[ws_bill_customer_sk@1, ws_quantity@2, ws_list_price@3, c_first_name@4, c_last_name@5] - │ CoalescePartitionsExec - │ [Stage 19] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 19] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[i_item_sk@0 as item_sk] │ FilterExec: count(Int64(1))@1 > 4, projection=[i_item_sk@0] │ ProjectionExec: expr=[i_item_sk@1 as i_item_sk, count(Int64(1))@3 as count(Int64(1))] │ AggregateExec: mode=FinalPartitioned, gby=[itemdesc@0 as itemdesc, i_item_sk@1 as i_item_sk, d_date@2 as d_date], aggr=[count(Int64(1))] │ [Stage 22] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 19 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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_bill_customer_sk@4, ws_quantity@5, ws_list_price@6, c_first_name@7, c_last_name@8] - │ CoalescePartitionsExec - │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ws_sold_date_sk@2 as ws_sold_date_sk, ws_item_sk@3 as ws_item_sk, ws_bill_customer_sk@4 as ws_bill_customer_sk, ws_quantity@5 as ws_quantity, ws_list_price@6 as ws_list_price, c_first_name@0 as c_first_name, c_last_name@1 as c_last_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@3, ws_bill_customer_sk@2)], projection=[c_first_name@1, c_last_name@2, ws_sold_date_sk@4, ws_item_sk@5, ws_bill_customer_sk@6, ws_quantity@7, ws_list_price@8] - │ CoalescePartitionsExec - │ [Stage 18] => 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_bill_customer_sk, ws_quantity, ws_list_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 19 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ 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_bill_customer_sk@4, ws_quantity@5, ws_list_price@6, c_first_name@7, c_last_name@8] + │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ ProjectionExec: expr=[ws_sold_date_sk@2 as ws_sold_date_sk, ws_item_sk@3 as ws_item_sk, ws_bill_customer_sk@4 as ws_bill_customer_sk, ws_quantity@5 as ws_quantity, ws_list_price@6 as ws_list_price, c_first_name@0 as c_first_name, c_last_name@1 as c_last_name] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@3, ws_bill_customer_sk@2)], projection=[c_first_name@1, c_last_name@2, ws_sold_date_sk@4, ws_item_sk@5, ws_bill_customer_sk@6, ws_quantity@7, ws_list_price@8] + │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_customer_sk, ws_quantity, ws_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 - │ 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 AND d_moy@2 = 2, projection=[d_date_sk@0] + ┌───── Stage 17 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 AND d_moy@2 = 2, projection=[d_date_sk@0] + │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 2000 AND d_moy@8 = 2, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= 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 (2000)] + └────────────────────────────────────────────────── + ┌───── Stage 18 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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/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 = 2000 AND d_moy@8 = 2, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= 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 (2000)] - └────────────────────────────────────────────────── - ┌───── Stage 18 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 + │ 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 22 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ RepartitionExec: partitioning=Hash([itemdesc@0, i_item_sk@1, d_date@2], 9), input_partitions=3 │ AggregateExec: mode=Partial, gby=[itemdesc@1 as itemdesc, i_item_sk@2 as i_item_sk, d_date@0 as d_date], aggr=[count(Int64(1))] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@1)], projection=[d_date@1, itemdesc@2, i_item_sk@3] - │ CoalescePartitionsExec - │ [Stage 21] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 21] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[substr(i_item_desc@1, 1, 30) as itemdesc, i_item_sk@0 as i_item_sk] │ 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_desc], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 21 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, d_date@0 as d_date] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_date@1, ss_item_sk@4] - │ CoalescePartitionsExec - │ [Stage 20] => 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], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 21 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, d_date@0 as d_date] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_date@1, ss_item_sk@4] + │ [Stage 20] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 20 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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)] - │ FilterExec: d_year@2 IN (SET) ([2000, 2001, 2002, 2003]), projection=[d_date_sk@0, d_date@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_date_sk, d_date, d_year], file_type=parquet, predicate=d_year@6 IN (SET) ([2000, 2001, 2002, 2003]), pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2003 AND 2003 <= d_year_max@1, required_guarantees=[d_year in (2000, 2001, 2002, 2003)] + ┌───── Stage 20 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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)] + │ FilterExec: d_year@2 IN (SET) ([2000, 2001, 2002, 2003]), projection=[d_date_sk@0, d_date@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_date_sk, d_date, d_year], file_type=parquet, predicate=d_year@6 IN (SET) ([2000, 2001, 2002, 2003]), pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2003 AND 2003 <= d_year_max@1, required_guarantees=[d_year in (2000, 2001, 2002, 2003)] └────────────────────────────────────────────────── ┌───── Stage 31 ── Tasks: t0:[p0..p8] │ RepartitionExec: partitioning=Hash([CAST(best_ss_customer.c_customer_sk AS Float64)@1], 9), input_partitions=3 @@ -2688,20 +2691,20 @@ mod tests { │ RepartitionExec: partitioning=Hash([c_customer_sk@0], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[c_customer_sk@2 as c_customer_sk], aggr=[sum(store_sales.ss_quantity * store_sales.ss_sales_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_sales_price@4, c_customer_sk@5] - │ CoalescePartitionsExec - │ [Stage 24] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 24] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[ss_sold_date_sk@1 as ss_sold_date_sk, ss_quantity@2 as ss_quantity, ss_sales_price@3 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@1)], projection=[c_customer_sk@0, ss_sold_date_sk@2, ss_quantity@4, ss_sales_price@5] │ [Stage 25] => NetworkShuffleExec: output_partitions=3, input_tasks=2 │ [Stage 26] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 24 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 IN (SET) ([2000, 2001, 2002, 2003]), projection=[d_date_sk@0] - │ 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_year], file_type=parquet, predicate=d_year@6 IN (SET) ([2000, 2001, 2002, 2003]), pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2003 AND 2003 <= d_year_max@1, required_guarantees=[d_year in (2000, 2001, 2002, 2003)] + ┌───── Stage 24 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 IN (SET) ([2000, 2001, 2002, 2003]), projection=[d_date_sk@0] + │ 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_year], file_type=parquet, predicate=d_year@6 IN (SET) ([2000, 2001, 2002, 2003]), pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2003 AND 2003 <= d_year_max@1, required_guarantees=[d_year in (2000, 2001, 2002, 2003)] └────────────────────────────────────────────────── ┌───── Stage 25 ── Tasks: t0:[p0..p8] t1:[p0..p8] │ RepartitionExec: partitioning=Hash([CAST(customer.c_customer_sk AS Float64)@1], 9), input_partitions=3 @@ -2738,10 +2741,11 @@ mod tests { │ SortExec: expr=[c_last_name@0 ASC NULLS LAST, c_first_name@1 ASC NULLS LAST, s_store_name@2 ASC NULLS LAST], preserve_partitioning=[false] │ ProjectionExec: expr=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, s_store_name@2 as s_store_name, sum(ssales.netpaid)@3 as paid] │ NestedLoopJoinExec: join_type=Inner, filter=join_proj_push_down_1@1 > Float64(0.05) * avg(ssales.netpaid)@0, projection=[c_last_name@0, c_first_name@1, s_store_name@2, sum(ssales.netpaid)@3, Float64(0.05) * avg(ssales.netpaid)@5] - │ ProjectionExec: expr=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, s_store_name@2 as s_store_name, sum(ssales.netpaid)@3 as sum(ssales.netpaid), CAST(sum(ssales.netpaid)@3 AS Decimal128(38, 15)) as join_proj_push_down_1] - │ CoalescePartitionsExec - │ 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], aggr=[sum(ssales.netpaid)] - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ ProjectionExec: expr=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, s_store_name@2 as s_store_name, sum(ssales.netpaid)@3 as sum(ssales.netpaid), CAST(sum(ssales.netpaid)@3 AS Decimal128(38, 15)) as join_proj_push_down_1] + │ CoalescePartitionsExec + │ 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], aggr=[sum(ssales.netpaid)] + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=2 │ 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 @@ -2758,49 +2762,49 @@ mod tests { │ 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 │ AggregateExec: mode=Partial, gby=[c_last_name@9 as c_last_name, c_first_name@8 as c_first_name, s_store_name@1 as s_store_name, ca_state@10 as ca_state, s_state@2 as s_state, i_color@5 as i_color, i_current_price@3 as i_current_price, i_manager_id@7 as i_manager_id, i_units@6 as i_units, i_size@4 as i_size], aggr=[sum(store_sales.ss_net_paid)], ordering_mode=PartiallySorted([5]) │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@9, ca_address_sk@0), (s_zip@3, ca_zip@2)], filter=c_birth_country@0 != CAST(upper(ca_country@1) AS Utf8View), projection=[ss_net_paid@0, s_store_name@1, s_state@2, i_current_price@4, i_size@5, i_color@6, i_units@7, i_manager_id@8, c_first_name@10, c_last_name@11, ca_state@14] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state, ca_zip, ca_country], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@5)], projection=[ss_net_paid@1, s_store_name@2, s_state@3, s_zip@4, i_current_price@5, i_size@6, i_color@7, i_units@8, i_manager_id@9, c_current_addr_sk@11, c_first_name@12, c_last_name@13, c_birth_country@14] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_birth_country@4 as c_birth_country, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ 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/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk, c_first_name, c_last_name, c_birth_country], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_customer_sk@1, ss_net_paid@2, s_store_name@3, s_state@4, s_zip@5, i_current_price@7, i_size@8, i_color@9, i_units@10, i_manager_id@11] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ FilterExec: i_color@3 = peach - │ 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_current_price, i_size, i_color, i_units, i_manager_id], file_type=parquet, predicate=i_color@17 = peach AND DynamicFilter [ empty ], pruning_predicate=i_color_null_count@2 != row_count@3 AND i_color_min@0 <= peach AND peach <= i_color_max@1, required_guarantees=[i_color in (peach)] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[ss_item_sk@3 as ss_item_sk, ss_customer_sk@4 as ss_customer_sk, ss_net_paid@5 as ss_net_paid, s_store_name@0 as s_store_name, s_state@1 as s_state, s_zip@2 as s_zip] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@4, ss_store_sk@2)], projection=[s_store_name@1, s_state@2, s_zip@3, ss_item_sk@5, ss_customer_sk@6, ss_net_paid@8] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_ticket_number@1, ss_ticket_number@3), (sr_item_sk@0, ss_item_sk@0)], projection=[ss_item_sk@2, ss_customer_sk@3, ss_store_sk@4, ss_net_paid@6] - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@5)], projection=[ss_net_paid@1, s_store_name@2, s_state@3, s_zip@4, i_current_price@5, i_size@6, i_color@7, i_units@8, i_manager_id@9, c_current_addr_sk@11, c_first_name@12, c_last_name@13, c_birth_country@14] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_birth_country@4 as c_birth_country, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] + │ 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/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk, c_first_name, c_last_name, c_birth_country], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_customer_sk@1, ss_net_paid@2, s_store_name@3, s_state@4, s_zip@5, i_current_price@7, i_size@8, i_color@9, i_units@10, i_manager_id@11] + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ FilterExec: i_color@3 = peach + │ 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_current_price, i_size, i_color, i_units, i_manager_id], file_type=parquet, predicate=i_color@17 = peach AND DynamicFilter [ empty ], pruning_predicate=i_color_null_count@2 != row_count@3 AND i_color_min@0 <= peach AND peach <= i_color_max@1, required_guarantees=[i_color in (peach)] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ss_item_sk@3 as ss_item_sk, ss_customer_sk@4 as ss_customer_sk, ss_net_paid@5 as ss_net_paid, s_store_name@0 as s_store_name, s_state@1 as s_state, s_zip@2 as s_zip] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@4, ss_store_sk@2)], projection=[s_store_name@1, s_state@2, s_zip@3, ss_item_sk@5, ss_customer_sk@6, ss_net_paid@8] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_ticket_number@1, ss_ticket_number@3), (sr_item_sk@0, ss_item_sk@0)], projection=[ss_item_sk@2, ss_customer_sk@3, ss_store_sk@4, ss_net_paid@6] + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, s_store_name@1 as s_store_name, s_state@2 as s_state, s_zip@3 as s_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_market_id@2 = 8, projection=[s_store_sk@0, s_store_name@1, s_state@3, s_zip@4] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_market_id, s_state, s_zip], file_type=parquet, predicate=s_market_id@10 = 8, pruning_predicate=s_market_id_null_count@2 != row_count@3 AND s_market_id_min@0 <= 8 AND 8 <= s_market_id_max@1, required_guarantees=[s_market_id in (8)] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, s_store_name@1 as s_store_name, s_state@2 as s_state, s_zip@3 as s_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_market_id@2 = 8, projection=[s_store_sk@0, s_store_name@1, s_state@3, s_zip@4] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_market_id, s_state, s_zip], file_type=parquet, predicate=s_market_id@10 = 8, pruning_predicate=s_market_id_null_count@2 != row_count@3 AND s_market_id_min@0 <= 8 AND 8 <= s_market_id_max@1, required_guarantees=[s_market_id in (8)] └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ RepartitionExec: partitioning=Hash([sr_ticket_number@1, sr_item_sk@0], 9), input_partitions=2 @@ -2822,48 +2826,48 @@ mod tests { │ 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 │ AggregateExec: mode=Partial, gby=[c_last_name@9 as c_last_name, c_first_name@8 as c_first_name, s_store_name@1 as s_store_name, ca_state@10 as ca_state, s_state@2 as s_state, i_color@5 as i_color, i_current_price@3 as i_current_price, i_manager_id@7 as i_manager_id, i_units@6 as i_units, i_size@4 as i_size], aggr=[sum(store_sales.ss_net_paid)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@9, ca_address_sk@0), (s_zip@3, ca_zip@2)], filter=c_birth_country@0 != CAST(upper(ca_country@1) AS Utf8View), projection=[ss_net_paid@0, s_store_name@1, s_state@2, i_current_price@4, i_size@5, i_color@6, i_units@7, i_manager_id@8, c_first_name@10, c_last_name@11, ca_state@14] - │ CoalescePartitionsExec - │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state, ca_zip, ca_country], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@5)], projection=[ss_net_paid@1, s_store_name@2, s_state@3, s_zip@4, i_current_price@5, i_size@6, i_color@7, i_units@8, i_manager_id@9, c_current_addr_sk@11, c_first_name@12, c_last_name@13, c_birth_country@14] - │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_birth_country@4 as c_birth_country, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ 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/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk, c_first_name, c_last_name, c_birth_country], file_type=parquet + ┌───── Stage 14 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@5)], projection=[ss_net_paid@1, s_store_name@2, s_state@3, s_zip@4, i_current_price@5, i_size@6, i_color@7, i_units@8, i_manager_id@9, c_current_addr_sk@11, c_first_name@12, c_last_name@13, c_birth_country@14] + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, c_birth_country@4 as c_birth_country, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] + │ 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/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk, c_first_name, c_last_name, c_birth_country], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_customer_sk@1, ss_net_paid@2, s_store_name@3, s_state@4, s_zip@5, i_current_price@7, i_size@8, i_color@9, i_units@10, i_manager_id@11] - │ CoalescePartitionsExec - │ [Stage 12] => 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_current_price, i_size, i_color, i_units, i_manager_id], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[ss_item_sk@3 as ss_item_sk, ss_customer_sk@4 as ss_customer_sk, ss_net_paid@5 as ss_net_paid, s_store_name@0 as s_store_name, s_state@1 as s_state, s_zip@2 as s_zip] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@4, ss_store_sk@2)], projection=[s_store_name@1, s_state@2, s_zip@3, ss_item_sk@5, ss_customer_sk@6, ss_net_paid@8] - │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_ticket_number@1, ss_ticket_number@3), (sr_item_sk@0, ss_item_sk@0)], projection=[ss_item_sk@2, ss_customer_sk@3, ss_store_sk@4, ss_net_paid@6] - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 13 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_customer_sk@1, ss_net_paid@2, s_store_name@3, s_state@4, s_zip@5, i_current_price@7, i_size@8, i_color@9, i_units@10, i_manager_id@11] + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_current_price, i_size, i_color, i_units, i_manager_id], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ss_item_sk@3 as ss_item_sk, ss_customer_sk@4 as ss_customer_sk, ss_net_paid@5 as ss_net_paid, s_store_name@0 as s_store_name, s_state@1 as s_state, s_zip@2 as s_zip] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@4, ss_store_sk@2)], projection=[s_store_name@1, s_state@2, s_zip@3, ss_item_sk@5, ss_customer_sk@6, ss_net_paid@8] + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_ticket_number@1, ss_ticket_number@3), (sr_item_sk@0, ss_item_sk@0)], projection=[ss_item_sk@2, ss_customer_sk@3, ss_store_sk@4, ss_net_paid@6] + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, s_store_name@1 as s_store_name, s_state@2 as s_state, s_zip@3 as s_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_market_id@2 = 8, projection=[s_store_sk@0, s_store_name@1, s_state@3, s_zip@4] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_market_id, s_state, s_zip], file_type=parquet, predicate=s_market_id@10 = 8, pruning_predicate=s_market_id_null_count@2 != row_count@3 AND s_market_id_min@0 <= 8 AND 8 <= s_market_id_max@1, required_guarantees=[s_market_id in (8)] + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, s_store_name@1 as s_store_name, s_state@2 as s_state, s_zip@3 as s_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_market_id@2 = 8, projection=[s_store_sk@0, s_store_name@1, s_state@3, s_zip@4] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_market_id, s_state, s_zip], file_type=parquet, predicate=s_market_id@10 = 8, pruning_predicate=s_market_id_null_count@2 != row_count@3 AND s_market_id_min@0 <= 8 AND 8 <= s_market_id_max@1, required_guarantees=[s_market_id in (8)] └────────────────────────────────────────────────── ┌───── Stage 10 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ RepartitionExec: partitioning=Hash([sr_ticket_number@1, sr_item_sk@0], 9), input_partitions=2 @@ -2896,60 +2900,60 @@ mod tests { │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, s_store_id@2, s_store_name@3], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[i_item_id@5 as i_item_id, i_item_desc@6 as i_item_desc, s_store_id@3 as s_store_id, s_store_name@4 as s_store_name], aggr=[sum(store_sales.ss_net_profit), sum(store_returns.sr_net_loss), sum(catalog_sales.cs_net_profit)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_net_profit@1, sr_net_loss@2, cs_net_profit@3, s_store_id@4, s_store_name@5, i_item_id@7, i_item_desc@8] - │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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, i_item_desc], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[ss_item_sk@2 as ss_item_sk, ss_net_profit@3 as ss_net_profit, sr_net_loss@4 as sr_net_loss, cs_net_profit@5 as cs_net_profit, s_store_id@0 as s_store_id, s_store_name@1 as s_store_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@1)], projection=[s_store_id@1, s_store_name@2, ss_item_sk@4, ss_net_profit@6, sr_net_loss@7, cs_net_profit@8] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_sold_date_sk@4, CAST(d3.d_date_sk AS Float64)@1)], projection=[ss_item_sk@0, ss_store_sk@1, ss_net_profit@2, sr_net_loss@3, cs_net_profit@5] - │ CoalescePartitionsExec - │ [Stage 8] => 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(d3.d_date_sk AS Float64)] - │ FilterExec: d_moy@2 >= 4 AND d_moy@2 <= 10 AND 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, d_moy], file_type=parquet, predicate=d_moy@8 >= 4 AND d_moy@8 <= 10 AND d_year@6 = 2001, pruning_predicate=d_moy_null_count@1 != row_count@2 AND d_moy_max@0 >= 4 AND d_moy_null_count@1 != row_count@2 AND d_moy_min@3 <= 10 AND d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_year in (2001)] - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, s_store_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(sr_returned_date_sk@3, CAST(d2.d_date_sk AS Float64)@1)], projection=[ss_item_sk@0, ss_store_sk@1, ss_net_profit@2, sr_net_loss@4, cs_sold_date_sk@5, cs_net_profit@6] - │ CoalescePartitionsExec - │ [Stage 7] => 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(d2.d_date_sk AS Float64)] - │ FilterExec: d_moy@2 >= 4 AND d_moy@2 <= 10 AND 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, d_moy], file_type=parquet, predicate=d_moy@8 >= 4 AND d_moy@8 <= 10 AND d_year@6 = 2001, pruning_predicate=d_moy_null_count@1 != row_count@2 AND d_moy_max@0 >= 4 AND d_moy_null_count@1 != row_count@2 AND d_moy_min@3 <= 10 AND d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_year in (2001)] - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_net_profit@5, sr_returned_date_sk@6, sr_net_loss@7, cs_sold_date_sk@8, cs_net_profit@9] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ss_sold_date_sk@2 as ss_sold_date_sk, ss_item_sk@3 as ss_item_sk, ss_store_sk@4 as ss_store_sk, ss_net_profit@5 as ss_net_profit, sr_returned_date_sk@6 as sr_returned_date_sk, sr_net_loss@7 as sr_net_loss, cs_sold_date_sk@0 as cs_sold_date_sk, cs_net_profit@1 as cs_net_profit] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(cs_bill_customer_sk@1, sr_customer_sk@6), (cs_item_sk@2, sr_item_sk@5)], projection=[cs_sold_date_sk@0, cs_net_profit@3, ss_sold_date_sk@4, ss_item_sk@5, ss_store_sk@6, ss_net_profit@7, sr_returned_date_sk@8, sr_net_loss@11] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ss_item_sk@2 as ss_item_sk, ss_net_profit@3 as ss_net_profit, sr_net_loss@4 as sr_net_loss, cs_net_profit@5 as cs_net_profit, s_store_id@0 as s_store_id, s_store_name@1 as s_store_name] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@1)], projection=[s_store_id@1, s_store_name@2, ss_item_sk@4, ss_net_profit@6, sr_net_loss@7, cs_net_profit@8] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_sold_date_sk@4, CAST(d3.d_date_sk AS Float64)@1)], projection=[ss_item_sk@0, ss_store_sk@1, ss_net_profit@2, sr_net_loss@3, cs_net_profit@5] + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] + │ FilterExec: d_moy@2 >= 4 AND d_moy@2 <= 10 AND 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, d_moy], file_type=parquet, predicate=d_moy@8 >= 4 AND d_moy@8 <= 10 AND d_year@6 = 2001, pruning_predicate=d_moy_null_count@1 != row_count@2 AND d_moy_max@0 >= 4 AND d_moy_null_count@1 != row_count@2 AND d_moy_min@3 <= 10 AND d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_year in (2001)] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, s_store_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(sr_returned_date_sk@3, CAST(d2.d_date_sk AS Float64)@1)], projection=[ss_item_sk@0, ss_store_sk@1, ss_net_profit@2, sr_net_loss@4, cs_sold_date_sk@5, cs_net_profit@6] + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] + │ FilterExec: d_moy@2 >= 4 AND d_moy@2 <= 10 AND 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, d_moy], file_type=parquet, predicate=d_moy@8 >= 4 AND d_moy@8 <= 10 AND d_year@6 = 2001, pruning_predicate=d_moy_null_count@1 != row_count@2 AND d_moy_max@0 >= 4 AND d_moy_null_count@1 != row_count@2 AND d_moy_min@3 <= 10 AND d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_year in (2001)] + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_net_profit@5, sr_returned_date_sk@6, sr_net_loss@7, cs_sold_date_sk@8, cs_net_profit@9] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ ProjectionExec: expr=[ss_sold_date_sk@2 as ss_sold_date_sk, ss_item_sk@3 as ss_item_sk, ss_store_sk@4 as ss_store_sk, ss_net_profit@5 as ss_net_profit, sr_returned_date_sk@6 as sr_returned_date_sk, sr_net_loss@7 as sr_net_loss, cs_sold_date_sk@0 as cs_sold_date_sk, cs_net_profit@1 as cs_net_profit] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(cs_bill_customer_sk@1, sr_customer_sk@6), (cs_item_sk@2, sr_item_sk@5)], projection=[cs_sold_date_sk@0, cs_net_profit@3, ss_sold_date_sk@4, ss_item_sk@5, ss_store_sk@6, ss_net_profit@7, sr_returned_date_sk@8, sr_net_loss@11] + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] - │ FilterExec: d_moy@2 = 4 AND d_year@1 = 2001, projection=[d_date_sk@0] - │ 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_year, d_moy], file_type=parquet, predicate=d_moy@8 = 4 AND d_year@6 = 2001, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 4 AND 4 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_moy in (4), d_year in (2001)] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] + │ FilterExec: d_moy@2 = 4 AND d_year@1 = 2001, projection=[d_date_sk@0] + │ 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_year, d_moy], file_type=parquet, predicate=d_moy@8 = 4 AND d_year@6 = 2001, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 4 AND 4 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_moy in (4), d_year in (2001)] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@1, cs_item_sk@2], 9), input_partitions=2 @@ -2994,39 +2998,39 @@ mod tests { │ RepartitionExec: partitioning=Hash([i_item_id@0], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[i_item_id@4 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)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(promotion.p_promo_sk AS Float64)@1, cs_promo_sk@0)], projection=[cs_quantity@3, cs_list_price@4, cs_sales_price@5, cs_coupon_amt@6, i_item_id@7] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, i_item_sk@0)], projection=[cs_promo_sk@1, cs_quantity@2, cs_list_price@3, cs_sales_price@4, cs_coupon_amt@5, i_item_id@7] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[p_promo_sk@0 as p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)] - │ FilterExec: p_channel_email@1 = N OR p_channel_event@2 = N, projection=[p_promo_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, p_channel_email, p_channel_event], file_type=parquet, predicate=p_channel_email@9 = N OR p_channel_event@14 = N, pruning_predicate=p_channel_email_null_count@2 != row_count@3 AND p_channel_email_min@0 <= N AND N <= p_channel_email_max@1 OR p_channel_event_null_count@6 != row_count@3 AND p_channel_event_min@4 <= N AND N <= p_channel_event_max@5, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_promo_sk@4, cs_quantity@5, cs_list_price@6, cs_sales_price@7, cs_coupon_amt@8] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, cs_bill_cdemo_sk@1)], projection=[cs_sold_date_sk@2, cs_item_sk@4, cs_promo_sk@5, cs_quantity@6, cs_list_price@7, cs_sales_price@8, cs_coupon_amt@9] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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,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 = 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 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[p_promo_sk@0 as p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)] + │ FilterExec: p_channel_email@1 = N OR p_channel_event@2 = N, projection=[p_promo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, p_channel_email, p_channel_event], file_type=parquet, predicate=p_channel_email@9 = N OR p_channel_event@14 = N, pruning_predicate=p_channel_email_null_count@2 != row_count@3 AND p_channel_email_min@0 <= N AND N <= p_channel_email_max@1 OR p_channel_event_null_count@6 != row_count@3 AND p_channel_event_min@4 <= N AND N <= p_channel_event_max@5, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ 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_promo_sk@4, cs_quantity@5, cs_list_price@6, cs_sales_price@7, cs_coupon_amt@8] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, cs_bill_cdemo_sk@1)], projection=[cs_sold_date_sk@2, cs_item_sk@4, cs_promo_sk@5, cs_quantity@6, cs_list_price@7, cs_sales_price@8, cs_coupon_amt@9] + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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,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 = 2000, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1, required_guarantees=[d_year in (2000)] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 9), input_partitions=2 @@ -3070,40 +3074,40 @@ mod tests { │ AggregateExec: mode=Partial, 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]) │ ProjectionExec: expr=[i_item_id@5 as i_item_id, s_state@4 as s_state, 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, s_state@5, i_item_id@7] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_quantity@2 as ss_quantity, ss_list_price@3 as ss_list_price, ss_sales_price@4 as ss_sales_price, ss_coupon_amt@5 as ss_coupon_amt, s_state@0 as s_state] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, ss_store_sk@1)], projection=[s_state@1, ss_item_sk@3, ss_quantity@5, ss_list_price@6, ss_sales_price@7, ss_coupon_amt@8] - │ CoalescePartitionsExec - │ [Stage 1] => 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, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_quantity@5, ss_list_price@6, ss_sales_price@7, ss_coupon_amt@8] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, ss_cdemo_sk@2)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_store_sk@5, ss_quantity@6, ss_list_price@7, ss_sales_price@8, ss_coupon_amt@9] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, s_state@1 as s_state, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_state@1 = TN - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_quantity@2 as ss_quantity, ss_list_price@3 as ss_list_price, ss_sales_price@4 as ss_sales_price, ss_coupon_amt@5 as ss_coupon_amt, s_state@0 as s_state] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, ss_store_sk@1)], projection=[s_state@1, ss_item_sk@3, ss_quantity@5, ss_list_price@6, ss_sales_price@7, ss_coupon_amt@8] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ 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_store_sk@4, ss_quantity@5, ss_list_price@6, ss_sales_price@7, ss_coupon_amt@8] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, ss_cdemo_sk@2)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_store_sk@5, ss_quantity@6, ss_list_price@7, ss_sales_price@8, ss_coupon_amt@9] + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, s_state@1 as s_state, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_state@1 = TN + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 = 2002, projection=[d_date_sk@0] - │ 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_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 = 2002, projection=[d_date_sk@0] + │ 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_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 9), input_partitions=2 @@ -3122,39 +3126,39 @@ mod tests { │ AggregateExec: mode=Partial, gby=[i_item_id@0 as i_item_id], aggr=[avg(results.agg1), avg(results.agg2), avg(results.agg3), avg(results.agg4)] │ ProjectionExec: expr=[i_item_id@4 as i_item_id, 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, i_item_id@6] - │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@1)], projection=[ss_item_sk@2, ss_quantity@4, ss_list_price@5, ss_sales_price@6, ss_coupon_amt@7] - │ CoalescePartitionsExec - │ [Stage 7] => 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, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_quantity@5, ss_list_price@6, ss_sales_price@7, ss_coupon_amt@8] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, ss_cdemo_sk@2)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_store_sk@5, ss_quantity@6, ss_list_price@7, ss_sales_price@8, ss_coupon_amt@9] - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_state@1 = TN, projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] + ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@1)], projection=[ss_item_sk@2, ss_quantity@4, ss_list_price@5, ss_sales_price@6, ss_coupon_amt@7] + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ 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_store_sk@4, ss_quantity@5, ss_list_price@6, ss_sales_price@7, ss_coupon_amt@8] + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, ss_cdemo_sk@2)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_store_sk@5, ss_quantity@6, ss_list_price@7, ss_sales_price@8, ss_coupon_amt@9] + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_state@1 = TN, projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 = 2002, projection=[d_date_sk@0] - │ 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_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 = 2002, projection=[d_date_sk@0] + │ 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_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] └────────────────────────────────────────────────── ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 9), input_partitions=2 @@ -3172,39 +3176,39 @@ mod tests { │ 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 + │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@1)], projection=[ss_item_sk@2, ss_quantity@4, ss_list_price@5, ss_sales_price@6, ss_coupon_amt@7] - │ CoalescePartitionsExec - │ [Stage 13] => 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, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_quantity@5, ss_list_price@6, ss_sales_price@7, ss_coupon_amt@8] - │ CoalescePartitionsExec - │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, ss_cdemo_sk@2)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_store_sk@5, ss_quantity@6, ss_list_price@7, ss_sales_price@8, ss_coupon_amt@9] - │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 16] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_state@1 = TN, projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] + ┌───── Stage 17 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@1)], projection=[ss_item_sk@2, ss_quantity@4, ss_list_price@5, ss_sales_price@6, ss_coupon_amt@7] + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ 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_store_sk@4, ss_quantity@5, ss_list_price@6, ss_sales_price@7, ss_coupon_amt@8] + │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, ss_cdemo_sk@2)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_store_sk@5, ss_quantity@6, ss_list_price@7, ss_sales_price@8, ss_coupon_amt@9] + │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 16] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 13 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_state@1 = TN, projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 = 2002, projection=[d_date_sk@0] - │ 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_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + ┌───── Stage 14 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 = 2002, projection=[d_date_sk@0] + │ 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_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] └────────────────────────────────────────────────── ┌───── Stage 15 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 9), input_partitions=2 @@ -3318,60 +3322,60 @@ mod tests { │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, s_store_id@2, s_store_name@3], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[i_item_id@5 as i_item_id, i_item_desc@6 as i_item_desc, s_store_id@3 as s_store_id, s_store_name@4 as s_store_name], aggr=[sum(store_sales.ss_quantity), sum(store_returns.sr_return_quantity), sum(catalog_sales.cs_quantity)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_quantity@1, sr_return_quantity@2, cs_quantity@3, s_store_id@4, s_store_name@5, i_item_id@7, i_item_desc@8] - │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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, i_item_desc], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[ss_item_sk@2 as ss_item_sk, ss_quantity@3 as ss_quantity, sr_return_quantity@4 as sr_return_quantity, cs_quantity@5 as cs_quantity, s_store_id@0 as s_store_id, s_store_name@1 as s_store_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@1)], projection=[s_store_id@1, s_store_name@2, ss_item_sk@4, ss_quantity@6, sr_return_quantity@7, cs_quantity@8] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_sold_date_sk@4, CAST(d3.d_date_sk AS Float64)@1)], projection=[ss_item_sk@0, ss_store_sk@1, ss_quantity@2, sr_return_quantity@3, cs_quantity@5] - │ CoalescePartitionsExec - │ [Stage 8] => 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(d3.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 1999 OR d_year@1 = 2000 OR 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 = 1999 OR d_year@6 = 2000 OR d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 OR 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 (1999, 2000, 2001)] - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, s_store_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d2.d_date_sk AS Float64)@1, sr_returned_date_sk@3)], projection=[ss_item_sk@2, ss_store_sk@3, ss_quantity@4, sr_return_quantity@6, cs_sold_date_sk@7, cs_quantity@8] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_quantity@5, sr_returned_date_sk@6, sr_return_quantity@7, cs_sold_date_sk@8, cs_quantity@9] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ss_sold_date_sk@2 as ss_sold_date_sk, ss_item_sk@3 as ss_item_sk, ss_store_sk@4 as ss_store_sk, ss_quantity@5 as ss_quantity, sr_returned_date_sk@6 as sr_returned_date_sk, sr_return_quantity@7 as sr_return_quantity, cs_sold_date_sk@0 as cs_sold_date_sk, cs_quantity@1 as cs_quantity] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(cs_bill_customer_sk@1, sr_customer_sk@6), (cs_item_sk@2, sr_item_sk@5)], projection=[cs_sold_date_sk@0, cs_quantity@3, ss_sold_date_sk@4, ss_item_sk@5, ss_store_sk@6, ss_quantity@7, sr_returned_date_sk@8, sr_return_quantity@11] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ss_item_sk@2 as ss_item_sk, ss_quantity@3 as ss_quantity, sr_return_quantity@4 as sr_return_quantity, cs_quantity@5 as cs_quantity, s_store_id@0 as s_store_id, s_store_name@1 as s_store_name] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@1)], projection=[s_store_id@1, s_store_name@2, ss_item_sk@4, ss_quantity@6, sr_return_quantity@7, cs_quantity@8] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_sold_date_sk@4, CAST(d3.d_date_sk AS Float64)@1)], projection=[ss_item_sk@0, ss_store_sk@1, ss_quantity@2, sr_return_quantity@3, cs_quantity@5] + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d3.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 1999 OR d_year@1 = 2000 OR 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 = 1999 OR d_year@6 = 2000 OR d_year@6 = 2001, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 OR 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 (1999, 2000, 2001)] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, s_store_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d2.d_date_sk AS Float64)@1, sr_returned_date_sk@3)], projection=[ss_item_sk@2, ss_store_sk@3, ss_quantity@4, sr_return_quantity@6, cs_sold_date_sk@7, cs_quantity@8] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_quantity@5, sr_returned_date_sk@6, sr_return_quantity@7, cs_sold_date_sk@8, cs_quantity@9] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ ProjectionExec: expr=[ss_sold_date_sk@2 as ss_sold_date_sk, ss_item_sk@3 as ss_item_sk, ss_store_sk@4 as ss_store_sk, ss_quantity@5 as ss_quantity, sr_returned_date_sk@6 as sr_returned_date_sk, sr_return_quantity@7 as sr_return_quantity, cs_sold_date_sk@0 as cs_sold_date_sk, cs_quantity@1 as cs_quantity] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(cs_bill_customer_sk@1, sr_customer_sk@6), (cs_item_sk@2, sr_item_sk@5)], projection=[cs_sold_date_sk@0, cs_quantity@3, ss_sold_date_sk@4, ss_item_sk@5, ss_store_sk@6, ss_quantity@7, sr_returned_date_sk@8, sr_return_quantity@11] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] - │ FilterExec: d_moy@2 >= 9 AND d_moy@2 <= 12 AND d_year@1 = 1999, projection=[d_date_sk@0] - │ 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_year, d_moy], file_type=parquet, predicate=d_moy@8 >= 9 AND d_moy@8 <= 12 AND d_year@6 = 1999, pruning_predicate=d_moy_null_count@1 != row_count@2 AND d_moy_max@0 >= 9 AND d_moy_null_count@1 != row_count@2 AND d_moy_min@3 <= 12 AND d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5, required_guarantees=[d_year in (1999)] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] + │ FilterExec: d_moy@2 >= 9 AND d_moy@2 <= 12 AND d_year@1 = 1999, projection=[d_date_sk@0] + │ 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_year, d_moy], file_type=parquet, predicate=d_moy@8 >= 9 AND d_moy@8 <= 12 AND d_year@6 = 1999, pruning_predicate=d_moy_null_count@1 != row_count@2 AND d_moy_max@0 >= 9 AND d_moy_null_count@1 != row_count@2 AND d_moy_min@3 <= 12 AND d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5, required_guarantees=[d_year in (1999)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] - │ FilterExec: d_moy@2 = 9 AND d_year@1 = 1999, projection=[d_date_sk@0] - │ 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_year, d_moy], file_type=parquet, predicate=d_moy@8 = 9 AND d_year@6 = 1999, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 9 AND 9 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5, required_guarantees=[d_moy in (9), d_year in (1999)] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] + │ FilterExec: d_moy@2 = 9 AND d_year@1 = 1999, projection=[d_date_sk@0] + │ 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_year, d_moy], file_type=parquet, predicate=d_moy@8 = 9 AND d_year@6 = 1999, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 9 AND 9 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5, required_guarantees=[d_moy in (9), d_year in (1999)] └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@1, cs_item_sk@2], 9), input_partitions=2 @@ -3418,222 +3422,222 @@ mod tests { │ 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 + │ [Stage 20] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 - │ 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, ca_county@5, web_sales@6, web_sales@8] - │ CoalescePartitionsExec - │ [Stage 16] => 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 19] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ ProjectionExec: expr=[ca_county@2 as ca_county, d_year@3 as d_year, store_sales@4 as store_sales, store_sales@5 as store_sales, store_sales@6 as store_sales, ca_county@0 as ca_county, web_sales@1 as web_sales] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_county@0, ca_county@0)] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_county@3, ca_county@0)], projection=[ca_county@0, d_year@1, store_sales@2, store_sales@4, store_sales@6] - │ CoalescePartitionsExec - │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[ca_county@0 as ca_county, sum(store_sales.ss_ext_sales_price)@3 as store_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(store_sales.ss_ext_sales_price)], ordering_mode=PartiallySorted([1, 2]) - │ [Stage 15] => 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 - │ 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 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 20 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ 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, ca_county@5, web_sales@6, web_sales@8] + │ [Stage 16] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 19] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 16 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ca_county@2 as ca_county, d_year@3 as d_year, store_sales@4 as store_sales, store_sales@5 as store_sales, store_sales@6 as store_sales, ca_county@0 as ca_county, web_sales@1 as web_sales] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_county@0, ca_county@0)] + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_county@3, ca_county@0)], projection=[ca_county@0, d_year@1, store_sales@2, store_sales@4, store_sales@6] + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, input_tasks=2 + │ ProjectionExec: expr=[ca_county@0 as ca_county, sum(store_sales.ss_ext_sales_price)@3 as store_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(store_sales.ss_ext_sales_price)], ordering_mode=PartiallySorted([1, 2]) + │ [Stage 15] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ 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 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([ca_county@0, d_qoy@1, d_year@2], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[ca_county@3 as ca_county, d_qoy@2 as d_qoy, d_year@1 as d_year], aggr=[sum(web_sales.ws_ext_sales_price)], ordering_mode=PartiallySorted([1, 2]) │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_bill_addr_sk@0, CAST(customer_address.ca_address_sk AS Float64)@2)], projection=[ws_ext_sales_price@1, d_year@2, d_qoy@3, ca_county@5] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_county@1 as ca_county, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_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/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_county], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ ProjectionExec: expr=[ws_bill_addr_sk@2 as ws_bill_addr_sk, ws_ext_sales_price@3 as ws_ext_sales_price, d_year@0 as d_year, d_qoy@1 as d_qoy] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ws_sold_date_sk@0)], projection=[d_year@1, d_qoy@2, ws_bill_addr_sk@5, ws_ext_sales_price@6] - │ CoalescePartitionsExec - │ [Stage 1] => 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_bill_addr_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ws_bill_addr_sk@2 as ws_bill_addr_sk, ws_ext_sales_price@3 as ws_ext_sales_price, d_year@0 as d_year, d_qoy@1 as d_qoy] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ws_sold_date_sk@0)], projection=[d_year@1, d_qoy@2, ws_bill_addr_sk@5, ws_ext_sales_price@6] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_addr_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_qoy@2 as d_qoy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_qoy@2 = 1 AND d_year@1 = 2000 - │ 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_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 1 AND d_year@6 = 2000, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 1 AND 1 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_qoy in (1), d_year in (2000)] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_qoy@2 as d_qoy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_qoy@2 = 1 AND d_year@1 = 2000 + │ 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_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 1 AND d_year@6 = 2000, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 1 AND 1 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_qoy in (1), d_year in (2000)] └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_county@0, ca_county@0)] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[ca_county@0 as ca_county, sum(store_sales.ss_ext_sales_price)@3 as store_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(store_sales.ss_ext_sales_price)], ordering_mode=PartiallySorted([1, 2]) - │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ ProjectionExec: expr=[ca_county@0 as ca_county, d_year@2 as d_year, sum(store_sales.ss_ext_sales_price)@3 as store_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(store_sales.ss_ext_sales_price)], ordering_mode=PartiallySorted([1, 2]) - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 12 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_county@0, ca_county@0)] + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, input_tasks=2 + │ ProjectionExec: expr=[ca_county@0 as ca_county, sum(store_sales.ss_ext_sales_price)@3 as store_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(store_sales.ss_ext_sales_price)], ordering_mode=PartiallySorted([1, 2]) + │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ca_county@0 as ca_county, d_year@2 as d_year, sum(store_sales.ss_ext_sales_price)@3 as store_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(store_sales.ss_ext_sales_price)], ordering_mode=PartiallySorted([1, 2]) + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([ca_county@0, d_qoy@1, d_year@2], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[ca_county@3 as ca_county, d_qoy@2 as d_qoy, d_year@1 as d_year], aggr=[sum(store_sales.ss_ext_sales_price)], ordering_mode=PartiallySorted([1, 2]) │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@0, CAST(customer_address.ca_address_sk AS Float64)@2)], projection=[ss_ext_sales_price@1, d_year@2, d_qoy@3, ca_county@5] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_county@1 as ca_county, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_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/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_county], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ ProjectionExec: expr=[ss_addr_sk@2 as ss_addr_sk, ss_ext_sales_price@3 as ss_ext_sales_price, d_year@0 as d_year, d_qoy@1 as d_qoy] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ss_sold_date_sk@0)], projection=[d_year@1, d_qoy@2, ss_addr_sk@5, ss_ext_sales_price@6] - │ 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_addr_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ss_addr_sk@2 as ss_addr_sk, ss_ext_sales_price@3 as ss_ext_sales_price, d_year@0 as d_year, d_qoy@1 as d_qoy] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ss_sold_date_sk@0)], projection=[d_year@1, d_qoy@2, ss_addr_sk@5, ss_ext_sales_price@6] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_addr_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_qoy@2 as d_qoy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_qoy@2 = 1 AND d_year@1 = 2000 - │ 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_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 1 AND d_year@6 = 2000, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 1 AND 1 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_qoy in (1), d_year in (2000)] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_qoy@2 as d_qoy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_qoy@2 = 1 AND d_year@1 = 2000 + │ 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_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 1 AND d_year@6 = 2000, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 1 AND 1 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_qoy in (1), d_year in (2000)] └────────────────────────────────────────────────── ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([ca_county@0, d_qoy@1, d_year@2], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[ca_county@3 as ca_county, d_qoy@2 as d_qoy, d_year@1 as d_year], aggr=[sum(store_sales.ss_ext_sales_price)], ordering_mode=PartiallySorted([1, 2]) │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@0, CAST(customer_address.ca_address_sk AS Float64)@2)], projection=[ss_ext_sales_price@1, d_year@2, d_qoy@3, ca_county@5] - │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_county@1 as ca_county, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_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/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_county], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ ProjectionExec: expr=[ss_addr_sk@2 as ss_addr_sk, ss_ext_sales_price@3 as ss_ext_sales_price, d_year@0 as d_year, d_qoy@1 as d_qoy] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ss_sold_date_sk@0)], projection=[d_year@1, d_qoy@2, ss_addr_sk@5, ss_ext_sales_price@6] - │ CoalescePartitionsExec - │ [Stage 9] => 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_addr_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ss_addr_sk@2 as ss_addr_sk, ss_ext_sales_price@3 as ss_ext_sales_price, d_year@0 as d_year, d_qoy@1 as d_qoy] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ss_sold_date_sk@0)], projection=[d_year@1, d_qoy@2, ss_addr_sk@5, ss_ext_sales_price@6] + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_addr_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_qoy@2 as d_qoy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_qoy@2 = 2 AND d_year@1 = 2000 - │ 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_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 2 AND d_year@6 = 2000, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 2 AND 2 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_qoy in (2), d_year in (2000)] + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_qoy@2 as d_qoy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_qoy@2 = 2 AND d_year@1 = 2000 + │ 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_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 2 AND d_year@6 = 2000, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 2 AND 2 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_qoy in (2), d_year in (2000)] └────────────────────────────────────────────────── ┌───── Stage 15 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([ca_county@0, d_qoy@1, d_year@2], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[ca_county@3 as ca_county, d_qoy@2 as d_qoy, d_year@1 as d_year], aggr=[sum(store_sales.ss_ext_sales_price)], ordering_mode=PartiallySorted([1, 2]) │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@0, CAST(customer_address.ca_address_sk AS Float64)@2)], projection=[ss_ext_sales_price@1, d_year@2, d_qoy@3, ca_county@5] - │ CoalescePartitionsExec - │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_county@1 as ca_county, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_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/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_county], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ ProjectionExec: expr=[ss_addr_sk@2 as ss_addr_sk, ss_ext_sales_price@3 as ss_ext_sales_price, d_year@0 as d_year, d_qoy@1 as d_qoy] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ss_sold_date_sk@0)], projection=[d_year@1, d_qoy@2, ss_addr_sk@5, ss_ext_sales_price@6] - │ 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/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_addr_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 14 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ss_addr_sk@2 as ss_addr_sk, ss_ext_sales_price@3 as ss_ext_sales_price, d_year@0 as d_year, d_qoy@1 as d_qoy] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ss_sold_date_sk@0)], projection=[d_year@1, d_qoy@2, ss_addr_sk@5, ss_ext_sales_price@6] + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_addr_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_qoy@2 as d_qoy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_qoy@2 = 3 AND d_year@1 = 2000 - │ 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_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 3 AND d_year@6 = 2000, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 3 AND 3 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_qoy in (3), d_year in (2000)] + ┌───── Stage 13 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_qoy@2 as d_qoy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_qoy@2 = 3 AND d_year@1 = 2000 + │ 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_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 3 AND d_year@6 = 2000, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 3 AND 3 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_qoy in (3), d_year in (2000)] └────────────────────────────────────────────────── ┌───── Stage 19 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([ca_county@0, d_qoy@1, d_year@2], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[ca_county@3 as ca_county, d_qoy@2 as d_qoy, d_year@1 as d_year], aggr=[sum(web_sales.ws_ext_sales_price)], ordering_mode=PartiallySorted([1, 2]) │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_bill_addr_sk@0, CAST(customer_address.ca_address_sk AS Float64)@2)], projection=[ws_ext_sales_price@1, d_year@2, d_qoy@3, ca_county@5] - │ CoalescePartitionsExec - │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_county@1 as ca_county, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_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/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_county], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 18 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ ProjectionExec: expr=[ws_bill_addr_sk@2 as ws_bill_addr_sk, ws_ext_sales_price@3 as ws_ext_sales_price, d_year@0 as d_year, d_qoy@1 as d_qoy] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ws_sold_date_sk@0)], projection=[d_year@1, d_qoy@2, ws_bill_addr_sk@5, ws_ext_sales_price@6] - │ CoalescePartitionsExec - │ [Stage 17] => 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_bill_addr_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 18 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ws_bill_addr_sk@2 as ws_bill_addr_sk, ws_ext_sales_price@3 as ws_ext_sales_price, d_year@0 as d_year, d_qoy@1 as d_qoy] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ws_sold_date_sk@0)], projection=[d_year@1, d_qoy@2, ws_bill_addr_sk@5, ws_ext_sales_price@6] + │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_addr_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 17 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_qoy@2 as d_qoy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_qoy@2 = 2 AND d_year@1 = 2000 - │ 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_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 2 AND d_year@6 = 2000, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 2 AND 2 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_qoy in (2), d_year in (2000)] + ┌───── Stage 17 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_qoy@2 as d_qoy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_qoy@2 = 2 AND d_year@1 = 2000 + │ 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_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 2 AND d_year@6 = 2000, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 2 AND 2 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_qoy in (2), d_year in (2000)] └────────────────────────────────────────────────── ┌───── Stage 23 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([ca_county@0, d_qoy@1, d_year@2], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[ca_county@3 as ca_county, d_qoy@2 as d_qoy, d_year@1 as d_year], aggr=[sum(web_sales.ws_ext_sales_price)], ordering_mode=PartiallySorted([1, 2]) │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_bill_addr_sk@0, CAST(customer_address.ca_address_sk AS Float64)@2)], projection=[ws_ext_sales_price@1, d_year@2, d_qoy@3, ca_county@5] - │ CoalescePartitionsExec - │ [Stage 22] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 22] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_county@1 as ca_county, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_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/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_county], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 22 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ ProjectionExec: expr=[ws_bill_addr_sk@2 as ws_bill_addr_sk, ws_ext_sales_price@3 as ws_ext_sales_price, d_year@0 as d_year, d_qoy@1 as d_qoy] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ws_sold_date_sk@0)], projection=[d_year@1, d_qoy@2, ws_bill_addr_sk@5, ws_ext_sales_price@6] - │ CoalescePartitionsExec - │ [Stage 21] => 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_bill_addr_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 22 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ws_bill_addr_sk@2 as ws_bill_addr_sk, ws_ext_sales_price@3 as ws_ext_sales_price, d_year@0 as d_year, d_qoy@1 as d_qoy] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ws_sold_date_sk@0)], projection=[d_year@1, d_qoy@2, ws_bill_addr_sk@5, ws_ext_sales_price@6] + │ [Stage 21] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_addr_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 21 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_qoy@2 as d_qoy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_qoy@2 = 3 AND d_year@1 = 2000 - │ 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_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 3 AND d_year@6 = 2000, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 3 AND 3 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_qoy in (3), d_year in (2000)] + ┌───── Stage 21 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_qoy@2 as d_qoy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_qoy@2 = 3 AND d_year@1 = 2000 + │ 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_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 3 AND d_year@6 = 2000, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 3 AND 3 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_qoy in (3), d_year in (2000)] └────────────────────────────────────────────────── "); Ok(()) @@ -3652,55 +3656,55 @@ mod tests { ┌───── 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 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ ProjectionExec: expr=[CAST(1.3 * CAST(avg(catalog_sales.cs_ext_discount_amt)@1 AS Float64) AS Decimal128(30, 15)) as Float64(1.3) * avg(catalog_sales.cs_ext_discount_amt), cs_item_sk@0 as cs_item_sk] - │ AggregateExec: mode=FinalPartitioned, gby=[cs_item_sk@0 as cs_item_sk], aggr=[avg(catalog_sales.cs_ext_discount_amt)] - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[CAST(1.3 * CAST(avg(catalog_sales.cs_ext_discount_amt)@1 AS Float64) AS Decimal128(30, 15)) as Float64(1.3) * avg(catalog_sales.cs_ext_discount_amt), cs_item_sk@0 as cs_item_sk] + │ AggregateExec: mode=FinalPartitioned, gby=[cs_item_sk@0 as cs_item_sk], aggr=[avg(catalog_sales.cs_ext_discount_amt)] + │ [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_item_sk@0], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[cs_item_sk@0 as cs_item_sk], aggr=[avg(catalog_sales.cs_ext_discount_amt)] │ 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_ext_discount_amt@4] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_date@1 >= 2000-01-27 AND d_date@1 <= 2000-04-26, projection=[d_date_sk@0] - │ 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, predicate=d_date@2 >= 2000-01-27 AND d_date@2 <= 2000-04-26, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-01-27 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-26, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_date@1 >= 2000-01-27 AND d_date@1 <= 2000-04-26, projection=[d_date_sk@0] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_date@1 >= 2000-01-27 AND d_date@1 <= 2000-04-26, projection=[d_date_sk@0] + │ 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, predicate=d_date@2 >= 2000-01-27 AND d_date@2 <= 2000-04-26, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-01-27 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-26, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_date@1 >= 2000-01-27 AND d_date@1 <= 2000-04-26, projection=[d_date_sk@0] + │ 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, predicate=d_date@2 >= 2000-01-27 AND d_date@2 <= 2000-04-26, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-01-27 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-26, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: i_manufact_id@1 = 977, projection=[i_item_sk@0] │ 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, predicate=d_date@2 >= 2000-01-27 AND d_date@2 <= 2000-04-26, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-01-27 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-26, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_manufact_id@1 = 977, projection=[i_item_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_manufact_id], file_type=parquet, predicate=i_manufact_id@13 = 977 AND DynamicFilter [ empty ], pruning_predicate=i_manufact_id_null_count@2 != row_count@3 AND i_manufact_id_min@0 <= 977 AND 977 <= i_manufact_id_max@1, required_guarantees=[i_manufact_id in (977)] + │ 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=i_manufact_id@13 = 977 AND DynamicFilter [ empty ], pruning_predicate=i_manufact_id_null_count@2 != row_count@3 AND i_manufact_id_min@0 <= 977 AND 977 <= i_manufact_id_max@1, required_guarantees=[i_manufact_id in (977)] └────────────────────────────────────────────────── "); Ok(()) @@ -3748,108 +3752,108 @@ mod tests { └────────────────────────────────────────────────── ┌───── 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 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[ss_item_sk@0, ss_ext_sales_price@2] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_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/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_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[ss_item_sk@0, ss_ext_sales_price@2] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] + │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_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/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_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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_addr_sk@4, ss_ext_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 1] => 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_addr_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ 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_addr_sk@4, ss_ext_sales_price@5] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_addr_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 = 1998 AND d_moy@2 = 5, projection=[d_date_sk@0] - │ 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_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 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 = 1998 AND d_moy@2 = 5, projection=[d_date_sk@0] + │ 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_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 + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_bill_addr_sk@0, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[cs_item_sk@1, cs_ext_sales_price@2] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_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/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_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_bill_addr_sk@0, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[cs_item_sk@1, cs_ext_sales_price@2] + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] + │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_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/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_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_bill_addr_sk@3, cs_item_sk@4, cs_ext_sales_price@5] - │ 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_bill_addr_sk, cs_item_sk, cs_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_bill_addr_sk@3, cs_item_sk@4, cs_ext_sales_price@5] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_addr_sk, cs_item_sk, cs_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 = 1998 AND d_moy@2 = 5, projection=[d_date_sk@0] - │ 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_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 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 = 1998 AND d_moy@2 = 5, projection=[d_date_sk@0] + │ 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_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 + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_bill_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[ws_item_sk@0, ws_ext_sales_price@2] - │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_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/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_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_bill_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[ws_item_sk@0, ws_ext_sales_price@2] + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] + │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_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/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_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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_bill_addr_sk@4, ws_ext_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 9] => 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_bill_addr_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ 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_bill_addr_sk@4, ws_ext_sales_price@5] + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_addr_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 = 1998 AND d_moy@2 = 5, projection=[d_date_sk@0] - │ 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_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 9 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 = 1998 AND d_moy@2 = 5, projection=[d_date_sk@0] + │ 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_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)] └────────────────────────────────────────────────── "); Ok(()) @@ -3866,58 +3870,58 @@ mod tests { │ 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 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 - │ ProjectionExec: expr=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk, count(Int64(1))@2 as cnt] - │ FilterExec: count(Int64(1))@2 >= 15 AND count(Int64(1))@2 <= 20 - │ AggregateExec: mode=FinalPartitioned, gby=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk], aggr=[count(Int64(1))] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 5 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk, count(Int64(1))@2 as cnt] + │ FilterExec: count(Int64(1))@2 >= 15 AND count(Int64(1))@2 <= 20 + │ AggregateExec: mode=FinalPartitioned, gby=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk], aggr=[count(Int64(1))] + │ [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_ticket_number@0, ss_customer_sk@1], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[ss_ticket_number@1 as ss_ticket_number, ss_customer_sk@0 as ss_customer_sk], aggr=[count(Int64(1))] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_customer_sk@2, ss_ticket_number@4] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@2)], projection=[ss_customer_sk@2, ss_hdemo_sk@3, ss_ticket_number@5] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_customer_sk@3, ss_hdemo_sk@4, ss_store_sk@5, ss_ticket_number@6] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_customer_sk, ss_hdemo_sk, ss_store_sk, ss_ticket_number], 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 - │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] - │ FilterExec: (hd_buy_potential@1 = >10000 OR hd_buy_potential@1 = Unknown) AND hd_vehicle_count@3 > 0 AND CASE WHEN hd_vehicle_count@3 > 0 THEN CAST(hd_dep_count@2 AS Float64) / CAST(hd_vehicle_count@3 AS Float64) END > 1.2, projection=[hd_demo_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_buy_potential, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=(hd_buy_potential@2 = >10000 OR hd_buy_potential@2 = Unknown) AND hd_vehicle_count@4 > 0 AND CASE WHEN hd_vehicle_count@4 > 0 THEN CAST(hd_dep_count@3 AS Float64) / CAST(hd_vehicle_count@4 AS Float64) END > 1.2, pruning_predicate=(hd_buy_potential_null_count@2 != row_count@3 AND hd_buy_potential_min@0 <= >10000 AND >10000 <= hd_buy_potential_max@1 OR hd_buy_potential_null_count@2 != row_count@3 AND hd_buy_potential_min@0 <= Unknown AND Unknown <= hd_buy_potential_max@1) AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_max@4 > 0, required_guarantees=[hd_buy_potential in (>10000, Unknown)] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_county@1 = Williamson County, projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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_county], file_type=parquet, predicate=s_county@23 = Williamson County, pruning_predicate=s_county_null_count@2 != row_count@3 AND s_county_min@0 <= Williamson County AND Williamson County <= s_county_max@1, required_guarantees=[s_county in (Williamson County)] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] + │ FilterExec: (hd_buy_potential@1 = >10000 OR hd_buy_potential@1 = Unknown) AND hd_vehicle_count@3 > 0 AND CASE WHEN hd_vehicle_count@3 > 0 THEN CAST(hd_dep_count@2 AS Float64) / CAST(hd_vehicle_count@3 AS Float64) END > 1.2, projection=[hd_demo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_buy_potential, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=(hd_buy_potential@2 = >10000 OR hd_buy_potential@2 = Unknown) AND hd_vehicle_count@4 > 0 AND CASE WHEN hd_vehicle_count@4 > 0 THEN CAST(hd_dep_count@3 AS Float64) / CAST(hd_vehicle_count@4 AS Float64) END > 1.2, pruning_predicate=(hd_buy_potential_null_count@2 != row_count@3 AND hd_buy_potential_min@0 <= >10000 AND >10000 <= hd_buy_potential_max@1 OR hd_buy_potential_null_count@2 != row_count@3 AND hd_buy_potential_min@0 <= Unknown AND Unknown <= hd_buy_potential_max@1) AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_max@4 > 0, required_guarantees=[hd_buy_potential in (>10000, Unknown)] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_county@1 = Williamson County, projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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_county], file_type=parquet, predicate=s_county@23 = Williamson County, pruning_predicate=s_county_null_count@2 != row_count@3 AND s_county_min@0 <= Williamson County AND Williamson County <= s_county_max@1, required_guarantees=[s_county in (Williamson County)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_dom@2 >= 1 AND d_dom@2 <= 3 OR d_dom@2 >= 25 AND d_dom@2 <= 28) AND (d_year@1 = 1999 OR d_year@1 = 2000 OR d_year@1 = 2001), projection=[d_date_sk@0] - │ 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_year, d_dom], file_type=parquet, predicate=(d_dom@9 >= 1 AND d_dom@9 <= 3 OR d_dom@9 >= 25 AND d_dom@9 <= 28) AND (d_year@6 = 1999 OR d_year@6 = 2000 OR d_year@6 = 2001), pruning_predicate=(d_dom_null_count@1 != row_count@2 AND d_dom_max@0 >= 1 AND d_dom_null_count@1 != row_count@2 AND d_dom_min@3 <= 3 OR d_dom_null_count@1 != row_count@2 AND d_dom_max@0 >= 25 AND d_dom_null_count@1 != row_count@2 AND d_dom_min@3 <= 28) AND (d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5 OR d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5 OR d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5), required_guarantees=[d_year in (1999, 2000, 2001)] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_dom@2 >= 1 AND d_dom@2 <= 3 OR d_dom@2 >= 25 AND d_dom@2 <= 28) AND (d_year@1 = 1999 OR d_year@1 = 2000 OR d_year@1 = 2001), projection=[d_date_sk@0] + │ 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_year, d_dom], file_type=parquet, predicate=(d_dom@9 >= 1 AND d_dom@9 <= 3 OR d_dom@9 >= 25 AND d_dom@9 <= 28) AND (d_year@6 = 1999 OR d_year@6 = 2000 OR d_year@6 = 2001), pruning_predicate=(d_dom_null_count@1 != row_count@2 AND d_dom_max@0 >= 1 AND d_dom_null_count@1 != row_count@2 AND d_dom_min@3 <= 3 OR d_dom_null_count@1 != row_count@2 AND d_dom_max@0 >= 25 AND d_dom_null_count@1 != row_count@2 AND d_dom_min@3 <= 28) AND (d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5 OR d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5 OR d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5), required_guarantees=[d_year in (1999, 2000, 2001)] └────────────────────────────────────────────────── "); Ok(()) @@ -3938,85 +3942,85 @@ mod tests { │ AggregateExec: mode=Partial, gby=[ca_state@0 as ca_state, cd_gender@1 as cd_gender, cd_marital_status@2 as cd_marital_status, cd_dep_count@3 as cd_dep_count, cd_dep_employed_count@4 as cd_dep_employed_count, cd_dep_college_count@5 as cd_dep_college_count], aggr=[count(Int64(1)), min(customer_demographics.cd_dep_count), max(customer_demographics.cd_dep_count), avg(customer_demographics.cd_dep_count), min(customer_demographics.cd_dep_employed_count), max(customer_demographics.cd_dep_employed_count), avg(customer_demographics.cd_dep_employed_count), min(customer_demographics.cd_dep_college_count), max(customer_demographics.cd_dep_college_count), avg(customer_demographics.cd_dep_college_count)] │ FilterExec: mark@6 OR mark@7, projection=[ca_state@0, cd_gender@1, cd_marital_status@2, cd_dep_count@3, cd_dep_employed_count@4, cd_dep_college_count@5] │ HashJoinExec: mode=CollectLeft, join_type=RightMark, on=[(cs_ship_customer_sk@0, CAST(c.c_customer_sk AS Float64)@8)], projection=[ca_state@1, cd_gender@2, cd_marital_status@3, cd_dep_count@4, cd_dep_employed_count@5, cd_dep_college_count@6, mark@7, mark@9] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, ca_state@1 as ca_state, cd_gender@2 as cd_gender, cd_marital_status@3 as cd_marital_status, cd_dep_count@4 as cd_dep_count, cd_dep_employed_count@5 as cd_dep_employed_count, cd_dep_college_count@6 as cd_dep_college_count, mark@7 as mark, CAST(c_customer_sk@0 AS Float64) as CAST(c.c_customer_sk AS Float64)] │ HashJoinExec: mode=CollectLeft, join_type=RightMark, on=[(ws_bill_customer_sk@0, CAST(c.c_customer_sk AS Float64)@7)], projection=[c_customer_sk@0, ca_state@1, cd_gender@2, cd_marital_status@3, cd_dep_count@4, cd_dep_employed_count@5, cd_dep_college_count@6, mark@8] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, ca_state@1 as ca_state, cd_gender@2 as cd_gender, cd_marital_status@3 as cd_marital_status, cd_dep_count@4 as cd_dep_count, cd_dep_employed_count@5 as cd_dep_employed_count, cd_dep_college_count@6 as cd_dep_college_count, CAST(c_customer_sk@0 AS Float64) as CAST(c.c_customer_sk AS Float64)] │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(ss_customer_sk@0, CAST(c.c_customer_sk AS Float64)@7)], projection=[c_customer_sk@0, ca_state@1, cd_gender@2, cd_marital_status@3, cd_dep_count@4, cd_dep_employed_count@5, cd_dep_college_count@6] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, ca_state@1 as ca_state, cd_gender@2 as cd_gender, cd_marital_status@3 as cd_marital_status, cd_dep_count@4 as cd_dep_count, cd_dep_employed_count@5 as cd_dep_employed_count, cd_dep_college_count@6 as cd_dep_college_count, CAST(c_customer_sk@0 AS Float64) as CAST(c.c_customer_sk AS Float64)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_cdemo_sk@1, CAST(customer_demographics.cd_demo_sk AS Float64)@6)], projection=[c_customer_sk@0, ca_state@2, cd_gender@4, cd_marital_status@5, cd_dep_count@6, cd_dep_employed_count@7, cd_dep_college_count@8] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_gender, cd_marital_status, cd_dep_count, cd_dep_employed_count, cd_dep_college_count, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_ship_customer_sk@3] - │ CoalescePartitionsExec - │ [Stage 1] => 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_ship_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 = 2002 AND d_qoy@2 < 4, projection=[d_date_sk@0] - │ 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_year, d_qoy], file_type=parquet, predicate=d_year@6 = 2002 AND d_qoy@10 < 4, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 AND d_qoy_null_count@5 != row_count@3 AND d_qoy_min@4 < 4, required_guarantees=[d_year in (2002)] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_bill_customer_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_date_sk, ws_bill_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 = 2002 AND d_qoy@2 < 4, projection=[d_date_sk@0] - │ 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_year, d_qoy], file_type=parquet, predicate=d_year@6 = 2002 AND d_qoy@10 < 4, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 AND d_qoy_null_count@5 != row_count@3 AND d_qoy_min@4 < 4, required_guarantees=[d_year in (2002)] - └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_customer_sk@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/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 = 2002 AND d_qoy@2 < 4, projection=[d_date_sk@0] - │ 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_year, d_qoy], file_type=parquet, predicate=d_year@6 = 2002 AND d_qoy@10 < 4, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 AND d_qoy_null_count@5 != row_count@3 AND d_qoy_min@4 < 4, required_guarantees=[d_year in (2002)] - └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[c_customer_sk@1 as c_customer_sk, c_current_cdemo_sk@2 as c_current_cdemo_sk, ca_state@0 as ca_state] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@2)], projection=[ca_state@1, c_customer_sk@2, c_current_cdemo_sk@3] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, 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/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_addr_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_ship_customer_sk@3] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_ship_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p3] t1:[p4..p7] - │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 = 2002 AND d_qoy@2 < 4, projection=[d_date_sk@0] + │ 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_year, d_qoy], file_type=parquet, predicate=d_year@6 = 2002 AND d_qoy@10 < 4, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 AND d_qoy_null_count@5 != row_count@3 AND d_qoy_min@4 < 4, required_guarantees=[d_year in (2002)] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_bill_customer_sk@3] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 = 2002 AND d_qoy@2 < 4, projection=[d_date_sk@0] + │ 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_year, d_qoy], file_type=parquet, predicate=d_year@6 = 2002 AND d_qoy@10 < 4, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 AND d_qoy_null_count@5 != row_count@3 AND d_qoy_min@4 < 4, required_guarantees=[d_year in (2002)] + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_customer_sk@3] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 = 2002 AND d_qoy@2 < 4, projection=[d_date_sk@0] + │ 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_year, d_qoy], file_type=parquet, predicate=d_year@6 = 2002 AND d_qoy@10 < 4, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1 AND d_qoy_null_count@5 != row_count@3 AND d_qoy_min@4 < 4, required_guarantees=[d_year in (2002)] + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[c_customer_sk@1 as c_customer_sk, c_current_cdemo_sk@2 as c_current_cdemo_sk, ca_state@0 as ca_state] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@2)], projection=[ca_state@1, c_customer_sk@2, c_current_cdemo_sk@3] + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_addr_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state], file_type=parquet └────────────────────────────────────────────────── "); Ok(()) @@ -4063,38 +4067,38 @@ mod tests { │ RepartitionExec: partitioning=Hash([i_category@0, i_class@1], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[i_category@3 as i_category, i_class@2 as i_class], aggr=[sum(store_sales.ss_net_profit), sum(store_sales.ss_ext_sales_price)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)], projection=[ss_ext_sales_price@3, ss_net_profit@4, i_class@5, i_category@6] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_store_sk@1, ss_ext_sales_price@2, ss_net_profit@3, i_class@5, i_category@6] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_class, i_category], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_state@1 = TN, projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_ext_sales_price@5, ss_net_profit@6] - │ 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/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_ext_sales_price, ss_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_state@1 = TN, projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_ext_sales_price@5, ss_net_profit@6] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_store_sk, ss_ext_sales_price, ss_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001, projection=[d_date_sk@0] - │ 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_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 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001, projection=[d_date_sk@0] + │ 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_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 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ RepartitionExec: partitioning=Hash([i_category@0], 3), input_partitions=3 @@ -4107,38 +4111,38 @@ mod tests { │ RepartitionExec: partitioning=Hash([i_category@0, i_class@1], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[i_category@3 as i_category, i_class@2 as i_class], aggr=[sum(store_sales.ss_net_profit), sum(store_sales.ss_ext_sales_price)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)], projection=[ss_ext_sales_price@3, ss_net_profit@4, i_class@5, i_category@6] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_store_sk@1, ss_ext_sales_price@2, ss_net_profit@3, i_class@5, i_category@6] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_class, i_category], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_state@1 = TN, projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_state@1 = TN, projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_ext_sales_price@5, ss_net_profit@6] - │ CoalescePartitionsExec - │ [Stage 6] => 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_store_sk, ss_ext_sales_price, ss_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_ext_sales_price@5, ss_net_profit@6] + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_store_sk, ss_ext_sales_price, ss_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001, projection=[d_date_sk@0] - │ 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_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..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001, projection=[d_date_sk@0] + │ 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_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)] @@ -4150,38 +4154,38 @@ mod tests { │ RepartitionExec: partitioning=Hash([i_category@0, i_class@1], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[i_category@3 as i_category, i_class@2 as i_class], aggr=[sum(store_sales.ss_net_profit), sum(store_sales.ss_ext_sales_price)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)], projection=[ss_ext_sales_price@3, ss_net_profit@4, i_class@5, i_category@6] - │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_store_sk@1, ss_ext_sales_price@2, ss_net_profit@3, i_class@5, i_category@6] - │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_class, i_category], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_state@1 = TN, projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] - └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_ext_sales_price@5, ss_net_profit@6] - │ CoalescePartitionsExec - │ [Stage 12] => 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_store_sk, ss_ext_sales_price, ss_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_state@1 = TN, projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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_state], file_type=parquet, predicate=s_state@24 = TN, pruning_predicate=s_state_null_count@2 != row_count@3 AND s_state_min@0 <= TN AND TN <= s_state_max@1, required_guarantees=[s_state in (TN)] + └────────────────────────────────────────────────── + ┌───── Stage 13 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_item_sk@3, ss_store_sk@4, ss_ext_sales_price@5, ss_net_profit@6] + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_store_sk, ss_ext_sales_price, ss_net_profit], 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 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001, projection=[d_date_sk@0] - │ 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_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 12 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001, projection=[d_date_sk@0] + │ 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_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)] └────────────────────────────────────────────────── "#); Ok(()) @@ -4203,36 +4207,36 @@ mod tests { │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, i_current_price@2], 6), input_partitions=2 │ AggregateExec: mode=Partial, 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=[] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@0)], projection=[i_item_id@1, i_item_desc@2, i_current_price@3] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_item_sk], 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 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(inv_date_sk@4, d_date_sk@0)], projection=[i_item_sk@0, i_item_id@1, i_item_desc@2, i_current_price@3] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ FilterExec: d_date@1 >= 2000-02-01 AND d_date@1 <= 2000-04-01, 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_date], file_type=parquet, predicate=d_date@2 >= 2000-02-01 AND d_date@2 <= 2000-04-01 AND DynamicFilter [ empty ], pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-02-01 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-01, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, inv_item_sk@1)], projection=[i_item_sk@0, i_item_id@1, i_item_desc@2, i_current_price@3, inv_date_sk@4] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ FilterExec: inv_quantity_on_hand@2 >= 100 AND inv_quantity_on_hand@2 <= 500, projection=[inv_date_sk@0, inv_item_sk@1] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/inventory/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], ...]}, projection=[inv_date_sk, inv_item_sk, inv_quantity_on_hand], file_type=parquet, predicate=inv_quantity_on_hand@3 >= 100 AND inv_quantity_on_hand@3 <= 500 AND DynamicFilter [ empty ], pruning_predicate=inv_quantity_on_hand_null_count@1 != row_count@2 AND inv_quantity_on_hand_max@0 >= 100 AND inv_quantity_on_hand_null_count@1 != row_count@2 AND inv_quantity_on_hand_min@3 <= 500, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_current_price@3 >= Some(6800),4,2 AND i_current_price@3 <= Some(9800),4,2 AND i_manufact_id@4 IN (SET) ([677, 940, 694, 808]), projection=[i_item_sk@0, i_item_id@1, i_item_desc@2, i_current_price@3] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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, i_item_desc, i_current_price, i_manufact_id], file_type=parquet, predicate=i_current_price@5 >= Some(6800),4,2 AND i_current_price@5 <= Some(9800),4,2 AND i_manufact_id@13 IN (SET) ([677, 940, 694, 808]) AND DynamicFilter [ empty ], pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 >= Some(6800),4,2 AND i_current_price_null_count@1 != row_count@2 AND i_current_price_min@3 <= Some(9800),4,2 AND (i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 677 AND 677 <= i_manufact_id_max@5 OR i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 940 AND 940 <= i_manufact_id_max@5 OR i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 694 AND 694 <= i_manufact_id_max@5 OR i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 808 AND 808 <= i_manufact_id_max@5), required_guarantees=[i_manufact_id in (677, 694, 808, 940)] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(inv_date_sk@4, d_date_sk@0)], projection=[i_item_sk@0, i_item_id@1, i_item_desc@2, i_current_price@3] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ FilterExec: d_date@1 >= 2000-02-01 AND d_date@1 <= 2000-04-01, 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_date], file_type=parquet, predicate=d_date@2 >= 2000-02-01 AND d_date@2 <= 2000-04-01 AND DynamicFilter [ empty ], pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-02-01 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-01, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, inv_item_sk@1)], projection=[i_item_sk@0, i_item_id@1, i_item_desc@2, i_current_price@3, inv_date_sk@4] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ FilterExec: inv_quantity_on_hand@2 >= 100 AND inv_quantity_on_hand@2 <= 500, projection=[inv_date_sk@0, inv_item_sk@1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/inventory/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], ...]}, projection=[inv_date_sk, inv_item_sk, inv_quantity_on_hand], file_type=parquet, predicate=inv_quantity_on_hand@3 >= 100 AND inv_quantity_on_hand@3 <= 500 AND DynamicFilter [ empty ], pruning_predicate=inv_quantity_on_hand_null_count@1 != row_count@2 AND inv_quantity_on_hand_max@0 >= 100 AND inv_quantity_on_hand_null_count@1 != row_count@2 AND inv_quantity_on_hand_min@3 <= 500, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: i_current_price@3 >= Some(6800),4,2 AND i_current_price@3 <= Some(9800),4,2 AND i_manufact_id@4 IN (SET) ([677, 940, 694, 808]), projection=[i_item_sk@0, i_item_id@1, i_item_desc@2, i_current_price@3] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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, i_item_desc, i_current_price, i_manufact_id], file_type=parquet, predicate=i_current_price@5 >= Some(6800),4,2 AND i_current_price@5 <= Some(9800),4,2 AND i_manufact_id@13 IN (SET) ([677, 940, 694, 808]) AND DynamicFilter [ empty ], pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 >= Some(6800),4,2 AND i_current_price_null_count@1 != row_count@2 AND i_current_price_min@3 <= Some(9800),4,2 AND (i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 677 AND 677 <= i_manufact_id_max@5 OR i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 940 AND 940 <= i_manufact_id_max@5 OR i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 694 AND 694 <= i_manufact_id_max@5 OR i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 808 AND 808 <= i_manufact_id_max@5), required_guarantees=[i_manufact_id in (677, 694, 808, 940)] └────────────────────────────────────────────────── "); Ok(()) @@ -4252,112 +4256,112 @@ mod tests { │ 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 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 - │ 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 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ 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 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1, d_date@2], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] │ ProjectionExec: expr=[c_last_name@2 as c_last_name, c_first_name@1 as c_first_name, d_date@0 as d_date] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_bill_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@3)], projection=[d_date@1, c_first_name@3, c_last_name@4] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ 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,__] t2:[__,__,__,p0] │ 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 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ ProjectionExec: expr=[ws_bill_customer_sk@1 as ws_bill_customer_sk, d_date@0 as d_date] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ws_sold_date_sk@0)], projection=[d_date@1, ws_bill_customer_sk@4] - │ CoalescePartitionsExec - │ [Stage 1] => 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_bill_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ws_bill_customer_sk@1 as ws_bill_customer_sk, d_date@0 as d_date] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ws_sold_date_sk@0)], projection=[d_date@1, ws_bill_customer_sk@4] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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)] - │ FilterExec: d_month_seq@2 >= 1200 AND d_month_seq@2 <= 1211, projection=[d_date_sk@0, d_date@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_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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)] + │ FilterExec: d_month_seq@2 >= 1200 AND d_month_seq@2 <= 1211, projection=[d_date_sk@0, d_date@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_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ 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 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 8 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ 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 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1, d_date@2], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] │ ProjectionExec: expr=[c_last_name@2 as c_last_name, c_first_name@1 as c_first_name, d_date@0 as d_date] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_bill_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@3)], projection=[d_date@1, c_first_name@3, c_last_name@4] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ 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,__] t2:[__,__,__,p0] │ 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..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ ProjectionExec: expr=[cs_bill_customer_sk@1 as cs_bill_customer_sk, d_date@0 as d_date] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, cs_sold_date_sk@0)], projection=[d_date@1, cs_bill_customer_sk@4] - │ 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_bill_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[cs_bill_customer_sk@1 as cs_bill_customer_sk, d_date@0 as d_date] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, cs_sold_date_sk@0)], projection=[d_date@1, cs_bill_customer_sk@4] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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)] - │ FilterExec: d_month_seq@2 >= 1200 AND d_month_seq@2 <= 1211, projection=[d_date_sk@0, d_date@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_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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)] + │ FilterExec: d_month_seq@2 >= 1200 AND d_month_seq@2 <= 1211, projection=[d_date_sk@0, d_date@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_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1, d_date@2], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] │ ProjectionExec: expr=[c_last_name@2 as c_last_name, c_first_name@1 as c_first_name, d_date@0 as d_date] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@3)], projection=[d_date@1, c_first_name@3, c_last_name@4] - │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ 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,__] t2:[__,__,__,p0] │ 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 10 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ ProjectionExec: expr=[ss_customer_sk@1 as ss_customer_sk, d_date@0 as d_date] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_date@1, ss_customer_sk@4] - │ CoalescePartitionsExec - │ [Stage 9] => 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_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ss_customer_sk@1 as ss_customer_sk, d_date@0 as d_date] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_date@1, ss_customer_sk@4] + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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)] - │ FilterExec: d_month_seq@2 >= 1200 AND d_month_seq@2 <= 1211, projection=[d_date_sk@0, d_date@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_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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)] + │ FilterExec: d_month_seq@2 >= 1200 AND d_month_seq@2 <= 1211, projection=[d_date_sk@0, d_date@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_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── "); Ok(()) @@ -4388,35 +4392,35 @@ mod tests { │ AggregateExec: mode=Partial, gby=[w_warehouse_name@3 as w_warehouse_name, w_warehouse_sk@2 as w_warehouse_sk, i_item_sk@1 as i_item_sk, d_moy@4 as d_moy], aggr=[stddev(inventory.inv_quantity_on_hand), avg(inventory.inv_quantity_on_hand)], ordering_mode=PartiallySorted([3]) │ ProjectionExec: expr=[inv_quantity_on_hand@1 as inv_quantity_on_hand, i_item_sk@2 as i_item_sk, w_warehouse_sk@3 as w_warehouse_sk, w_warehouse_name@4 as w_warehouse_name, d_moy@0 as d_moy] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_date_sk@0, inv_date_sk@0)], projection=[d_moy@1, inv_quantity_on_hand@3, i_item_sk@4, w_warehouse_sk@5, w_warehouse_name@6] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[inv_date_sk@2 as inv_date_sk, inv_quantity_on_hand@3 as inv_quantity_on_hand, i_item_sk@4 as i_item_sk, w_warehouse_sk@0 as w_warehouse_sk, w_warehouse_name@1 as w_warehouse_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(w_warehouse_sk@0, inv_warehouse_sk@1)], projection=[w_warehouse_sk@0, w_warehouse_name@1, inv_date_sk@2, inv_quantity_on_hand@4, i_item_sk@5] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[inv_date_sk@1 as inv_date_sk, inv_warehouse_sk@2 as inv_warehouse_sk, inv_quantity_on_hand@3 as inv_quantity_on_hand, i_item_sk@0 as i_item_sk] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, inv_item_sk@1)], projection=[i_item_sk@0, inv_date_sk@1, inv_warehouse_sk@3, inv_quantity_on_hand@4] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/inventory/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], ...]}, projection=[inv_date_sk, inv_item_sk, inv_warehouse_sk, inv_quantity_on_hand], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: d_moy@2 = 1 AND d_year@1 = 2001, projection=[d_date_sk@0, d_moy@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/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 1 AND d_year@6 = 2001, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 1 AND 1 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_moy in (1), d_year in (2001)] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: d_moy@2 = 1 AND d_year@1 = 2001, projection=[d_date_sk@0, d_moy@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/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 1 AND d_year@6 = 2001, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 1 AND 1 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_moy in (1), d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_warehouse_name], file_type=parquet + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_warehouse_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk], file_type=parquet + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ RepartitionExec: partitioning=Hash([i_item_sk@1, w_warehouse_sk@0], 3), input_partitions=3 @@ -4431,35 +4435,35 @@ mod tests { │ AggregateExec: mode=Partial, gby=[w_warehouse_name@3 as w_warehouse_name, w_warehouse_sk@2 as w_warehouse_sk, i_item_sk@1 as i_item_sk, d_moy@4 as d_moy], aggr=[stddev(inventory.inv_quantity_on_hand), avg(inventory.inv_quantity_on_hand)], ordering_mode=PartiallySorted([3]) │ ProjectionExec: expr=[inv_quantity_on_hand@1 as inv_quantity_on_hand, i_item_sk@2 as i_item_sk, w_warehouse_sk@3 as w_warehouse_sk, w_warehouse_name@4 as w_warehouse_name, d_moy@0 as d_moy] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_date_sk@0, inv_date_sk@0)], projection=[d_moy@1, inv_quantity_on_hand@3, i_item_sk@4, w_warehouse_sk@5, w_warehouse_name@6] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[inv_date_sk@2 as inv_date_sk, inv_quantity_on_hand@3 as inv_quantity_on_hand, i_item_sk@4 as i_item_sk, w_warehouse_sk@0 as w_warehouse_sk, w_warehouse_name@1 as w_warehouse_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(w_warehouse_sk@0, inv_warehouse_sk@1)], projection=[w_warehouse_sk@0, w_warehouse_name@1, inv_date_sk@2, inv_quantity_on_hand@4, i_item_sk@5] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[inv_date_sk@1 as inv_date_sk, inv_warehouse_sk@2 as inv_warehouse_sk, inv_quantity_on_hand@3 as inv_quantity_on_hand, i_item_sk@0 as i_item_sk] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, inv_item_sk@1)], projection=[i_item_sk@0, inv_date_sk@1, inv_warehouse_sk@3, inv_quantity_on_hand@4] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/inventory/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], ...]}, projection=[inv_date_sk, inv_item_sk, inv_warehouse_sk, inv_quantity_on_hand], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: d_moy@2 = 2 AND d_year@1 = 2001, projection=[d_date_sk@0, d_moy@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/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 2 AND d_year@6 = 2001, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 2 AND 2 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_moy in (2), d_year in (2001)] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: d_moy@2 = 2 AND d_year@1 = 2001, projection=[d_date_sk@0, d_moy@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/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 2 AND d_year@6 = 2001, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 2 AND 2 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_moy in (2), d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_warehouse_name], file_type=parquet + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_warehouse_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk], file_type=parquet + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk], file_type=parquet └────────────────────────────────────────────────── "); Ok(()) @@ -4483,39 +4487,39 @@ mod tests { │ AggregateExec: mode=Partial, gby=[w_state@3 as w_state, i_item_id@4 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)] │ ProjectionExec: expr=[d_date@4 as __common_expr_1, cs_sales_price@0 as cs_sales_price, cr_refunded_cash@1 as cr_refunded_cash, w_state@2 as w_state, i_item_id@3 as i_item_id] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@2)], projection=[cs_sales_price@1, cr_refunded_cash@2, w_state@3, i_item_id@4, d_date@6] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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(date_dim.d_date_sk AS Float64)] │ FilterExec: d_date@1 >= 2000-02-10 AND d_date@1 <= 2000-04-10 │ 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, predicate=d_date@2 >= 2000-02-10 AND d_date@2 <= 2000-04-10, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-02-10 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-10, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@1, i_item_sk@0)], projection=[cs_sold_date_sk@0, cs_sales_price@2, cr_refunded_cash@3, w_state@4, i_item_id@6] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ FilterExec: i_current_price@2 >= Some(99),4,2 AND i_current_price@2 <= Some(149),4,2, projection=[i_item_sk@0, i_item_id@1] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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, i_current_price], file_type=parquet, predicate=i_current_price@5 >= Some(99),4,2 AND i_current_price@5 <= Some(149),4,2 AND DynamicFilter [ empty ], pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 >= Some(99),4,2 AND i_current_price_null_count@1 != row_count@2 AND i_current_price_min@3 <= Some(149),4,2, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_item_sk@2 as cs_item_sk, cs_sales_price@3 as cs_sales_price, cr_refunded_cash@4 as cr_refunded_cash, w_state@0 as w_state] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(warehouse.w_warehouse_sk AS Float64)@2, cs_warehouse_sk@1)], projection=[w_state@1, cs_sold_date_sk@3, cs_item_sk@5, cs_sales_price@6, cr_refunded_cash@7] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_warehouse_sk@2 as cs_warehouse_sk, cs_item_sk@3 as cs_item_sk, cs_sales_price@4 as cs_sales_price, cr_refunded_cash@0 as cr_refunded_cash] - │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(cr_order_number@1, cs_order_number@3), (cr_item_sk@0, cs_item_sk@2)], projection=[cr_refunded_cash@2, cs_sold_date_sk@3, cs_warehouse_sk@4, cs_item_sk@5, cs_sales_price@7] - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_state, CAST(w_warehouse_sk@0 AS Float64) as CAST(warehouse.w_warehouse_sk AS Float64)], file_type=parquet + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@1, i_item_sk@0)], projection=[cs_sold_date_sk@0, cs_sales_price@2, cr_refunded_cash@3, w_state@4, i_item_id@6] + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ FilterExec: i_current_price@2 >= Some(99),4,2 AND i_current_price@2 <= Some(149),4,2, projection=[i_item_sk@0, i_item_id@1] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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, i_current_price], file_type=parquet, predicate=i_current_price@5 >= Some(99),4,2 AND i_current_price@5 <= Some(149),4,2 AND DynamicFilter [ empty ], pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 >= Some(99),4,2 AND i_current_price_null_count@1 != row_count@2 AND i_current_price_min@3 <= Some(149),4,2, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_item_sk@2 as cs_item_sk, cs_sales_price@3 as cs_sales_price, cr_refunded_cash@4 as cr_refunded_cash, w_state@0 as w_state] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(warehouse.w_warehouse_sk AS Float64)@2, cs_warehouse_sk@1)], projection=[w_state@1, cs_sold_date_sk@3, cs_item_sk@5, cs_sales_price@6, cr_refunded_cash@7] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_warehouse_sk@2 as cs_warehouse_sk, cs_item_sk@3 as cs_item_sk, cs_sales_price@4 as cs_sales_price, cr_refunded_cash@0 as cr_refunded_cash] + │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(cr_order_number@1, cs_order_number@3), (cr_item_sk@0, cs_item_sk@2)], projection=[cr_refunded_cash@2, cs_sold_date_sk@3, cs_warehouse_sk@4, cs_item_sk@5, cs_sales_price@7] + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_state, CAST(w_warehouse_sk@0 AS Float64) as CAST(warehouse.w_warehouse_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ RepartitionExec: partitioning=Hash([cr_order_number@1, cr_item_sk@0], 9), input_partitions=2 @@ -4546,18 +4550,18 @@ mod tests { │ FilterExec: CASE WHEN __always_true@2 IS NULL THEN 0 ELSE item_cnt@1 END > 0, projection=[i_product_name@0] │ ProjectionExec: expr=[i_product_name@2 as i_product_name, item_cnt@0 as item_cnt, __always_true@1 as __always_true] │ HashJoinExec: mode=CollectLeft, join_type=Right, on=[(i_manufact@1, i_manufact@0)], projection=[item_cnt@0, __always_true@2, i_product_name@4] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=1 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, input_tasks=1 │ FilterExec: i_manufact_id@0 >= 738 AND i_manufact_id@0 <= 778, projection=[i_manufact@1, i_product_name@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/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_manufact_id, i_manufact, i_product_name], file_type=parquet, predicate=i_manufact_id@13 >= 738 AND i_manufact_id@13 <= 778, pruning_predicate=i_manufact_id_null_count@1 != row_count@2 AND i_manufact_id_max@0 >= 738 AND i_manufact_id_null_count@1 != row_count@2 AND i_manufact_id_min@3 <= 778, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ ProjectionExec: expr=[count(Int64(1))@1 as item_cnt, i_manufact@0 as i_manufact, true as __always_true] - │ AggregateExec: mode=FinalPartitioned, gby=[i_manufact@0 as i_manufact], aggr=[count(Int64(1))] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + ┌───── Stage 2 ── Tasks: t0:[p0..p1] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[count(Int64(1))@1 as item_cnt, i_manufact@0 as i_manufact, true as __always_true] + │ AggregateExec: mode=FinalPartitioned, gby=[i_manufact@0 as i_manufact], aggr=[count(Int64(1))] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ RepartitionExec: partitioning=Hash([i_manufact@0], 3), input_partitions=3 @@ -4588,28 +4592,28 @@ mod tests { │ RepartitionExec: partitioning=Hash([d_year@0, i_category_id@1, i_category@2], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[d_year@0 as d_year, i_category_id@2 as i_category_id, i_category@3 as i_category], aggr=[sum(store_sales.ss_ext_sales_price)], ordering_mode=PartiallySorted([0]) │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@1, i_item_sk@0)], projection=[d_year@0, ss_ext_sales_price@2, i_category_id@4, i_category@5] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ FilterExec: i_manager_id@3 = 1, projection=[i_item_sk@0, i_category_id@1, i_category@2] │ 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_id, i_category, i_manager_id], file_type=parquet, predicate=i_manager_id@20 = 1 AND DynamicFilter [ empty ], pruning_predicate=i_manager_id_null_count@2 != row_count@3 AND i_manager_id_min@0 <= 1 AND 1 <= i_manager_id_max@1, required_guarantees=[i_manager_id in (1)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(dt.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_ext_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 1] => 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_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(dt.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_ext_sales_price@5] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(dt.d_date_sk AS Float64)] - │ FilterExec: d_moy@2 = 11 AND d_year@1 = 2000, projection=[d_date_sk@0, d_year@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_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 11 AND d_year@6 = 2000, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_moy in (11), d_year in (2000)] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(dt.d_date_sk AS Float64)] + │ FilterExec: d_moy@2 = 11 AND d_year@1 = 2000, projection=[d_date_sk@0, d_year@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_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 11 AND d_year@6 = 2000, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_moy in (11), d_year in (2000)] └────────────────────────────────────────────────── "); Ok(()) @@ -4633,29 +4637,29 @@ mod tests { │ AggregateExec: mode=Partial, gby=[s_store_name@3 as s_store_name, s_store_id@2 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)] │ ProjectionExec: expr=[d_day_name@2 as d_day_name, ss_sales_price@3 as ss_sales_price, s_store_id@0 as s_store_id, s_store_name@1 as s_store_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@1)], projection=[s_store_id@1, s_store_name@2, d_day_name@4, ss_sales_price@6] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_day_name@1, ss_store_sk@4, ss_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_store_sk, ss_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 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, s_store_id@1 as s_store_id, s_store_name@2 as s_store_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_gmt_offset@3 = Some(-500),3,2, projection=[s_store_sk@0, s_store_id@1, s_store_name@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/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, s_store_name, 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 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, s_store_id@1 as s_store_id, s_store_name@2 as s_store_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_gmt_offset@3 = Some(-500),3,2, projection=[s_store_sk@0, s_store_id@1, s_store_name@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/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, s_store_name, 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 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_day_name@1 as d_day_name, 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, d_day_name@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/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_day_name], 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 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_day_name@1 as d_day_name, 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, d_day_name@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/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_day_name], 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)] └────────────────────────────────────────────────── "#); Ok(()) @@ -4672,25 +4676,25 @@ mod tests { │ 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 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 - │ 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_product_name], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ 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_product_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p4..p7] - │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 - │ 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_product_name], file_type=parquet + ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ 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_product_name], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p5] │ RepartitionExec: partitioning=Hash([rnk@1], 6), input_partitions=3 @@ -4781,38 +4785,38 @@ mod tests { └────────────────────────────────────────────────── ┌───── 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 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ 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_sales_price@4, ca_city@5, ca_zip@6] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ws_sold_date_sk@2 as ws_sold_date_sk, ws_item_sk@3 as ws_item_sk, ws_sales_price@4 as ws_sales_price, ca_city@0 as ca_city, ca_zip@1 as ca_zip] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@3)], projection=[ca_city@1, ca_zip@2, ws_sold_date_sk@3, ws_item_sk@4, ws_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[ws_sold_date_sk@1 as ws_sold_date_sk, ws_item_sk@2 as ws_item_sk, ws_sales_price@3 as ws_sales_price, c_current_addr_sk@0 as c_current_addr_sk] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@2, ws_bill_customer_sk@2)], projection=[c_current_addr_sk@1, ws_sold_date_sk@3, ws_item_sk@4, ws_sales_price@6] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_qoy@2 = 2 AND d_year@1 = 2001, projection=[d_date_sk@0] - │ 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_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 2 AND d_year@6 = 2001, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 2 AND 2 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_qoy in (2), d_year in (2001)] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ 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_sales_price@4, ca_city@5, ca_zip@6] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ ProjectionExec: expr=[ws_sold_date_sk@2 as ws_sold_date_sk, ws_item_sk@3 as ws_item_sk, ws_sales_price@4 as ws_sales_price, ca_city@0 as ca_city, ca_zip@1 as ca_zip] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@3)], projection=[ca_city@1, ca_zip@2, ws_sold_date_sk@3, ws_item_sk@4, ws_sales_price@5] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ ProjectionExec: expr=[ws_sold_date_sk@1 as ws_sold_date_sk, ws_item_sk@2 as ws_item_sk, ws_sales_price@3 as ws_sales_price, c_current_addr_sk@0 as c_current_addr_sk] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@2, ws_bill_customer_sk@2)], projection=[c_current_addr_sk@1, ws_sold_date_sk@3, ws_item_sk@4, ws_sales_price@6] + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_qoy@2 = 2 AND d_year@1 = 2001, projection=[d_date_sk@0] + │ 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_year, d_qoy], file_type=parquet, predicate=d_qoy@10 = 2 AND d_year@6 = 2001, pruning_predicate=d_qoy_null_count@2 != row_count@3 AND d_qoy_min@0 <= 2 AND 2 <= d_qoy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5, required_guarantees=[d_qoy in (2), d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_city, ca_zip], file_type=parquet + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_city, ca_zip], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p0..p8] │ RepartitionExec: partitioning=Hash([CAST(customer.c_customer_sk AS Float64)@2], 9), input_partitions=3 @@ -4841,76 +4845,76 @@ mod tests { │ 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 + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@4)], projection=[ss_ticket_number@0, bought_city@2, amt@3, profit@4, c_current_addr_sk@6, c_first_name@7, c_last_name@8] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk, c_first_name, c_last_name], file_type=parquet + ┌───── Stage 7 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@4)], projection=[ss_ticket_number@0, bought_city@2, amt@3, profit@4, c_current_addr_sk@6, c_first_name@7, c_last_name@8] + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, input_tasks=2 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk, c_first_name, c_last_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ ProjectionExec: expr=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk, ca_city@3 as bought_city, sum(store_sales.ss_coupon_amt)@4 as amt, sum(store_sales.ss_net_profit)@5 as profit] - │ AggregateExec: mode=FinalPartitioned, gby=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk, ss_addr_sk@2 as ss_addr_sk, ca_city@3 as ca_city], aggr=[sum(store_sales.ss_coupon_amt), sum(store_sales.ss_net_profit)] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 6 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk, ca_city@3 as bought_city, sum(store_sales.ss_coupon_amt)@4 as amt, sum(store_sales.ss_net_profit)@5 as profit] + │ AggregateExec: mode=FinalPartitioned, gby=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk, ss_addr_sk@2 as ss_addr_sk, ca_city@3 as ca_city], aggr=[sum(store_sales.ss_coupon_amt), sum(store_sales.ss_net_profit)] + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([ss_ticket_number@0, ss_customer_sk@1, ss_addr_sk@2, ca_city@3], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[ss_ticket_number@2 as ss_ticket_number, ss_customer_sk@0 as ss_customer_sk, ss_addr_sk@1 as ss_addr_sk, ca_city@5 as ca_city], aggr=[sum(store_sales.ss_coupon_amt), sum(store_sales.ss_net_profit)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@2)], projection=[ss_customer_sk@0, ss_addr_sk@1, ss_ticket_number@2, ss_coupon_amt@3, ss_net_profit@4, ca_city@6] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_city@1 as ca_city, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_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/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 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_customer_sk@2, ss_addr_sk@4, ss_ticket_number@5, ss_coupon_amt@6, ss_net_profit@7] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@3)], projection=[ss_customer_sk@2, ss_hdemo_sk@3, ss_addr_sk@4, ss_ticket_number@6, ss_coupon_amt@7, ss_net_profit@8] - │ 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, ss_sold_date_sk@0)], projection=[ss_customer_sk@3, ss_hdemo_sk@4, ss_addr_sk@5, ss_store_sk@6, ss_ticket_number@7, ss_coupon_amt@8, ss_net_profit@9] - │ 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_date_sk, ss_customer_sk, ss_hdemo_sk, ss_addr_sk, ss_store_sk, ss_ticket_number, ss_coupon_amt, ss_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_customer_sk@2, ss_addr_sk@4, ss_ticket_number@5, ss_coupon_amt@6, ss_net_profit@7] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@3)], projection=[ss_customer_sk@2, ss_hdemo_sk@3, ss_addr_sk@4, ss_ticket_number@6, ss_coupon_amt@7, ss_net_profit@8] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_customer_sk@3, ss_hdemo_sk@4, ss_addr_sk@5, ss_store_sk@6, ss_ticket_number@7, ss_coupon_amt@8, ss_net_profit@9] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_customer_sk, ss_hdemo_sk, ss_addr_sk, ss_store_sk, ss_ticket_number, ss_coupon_amt, ss_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 - │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] - │ FilterExec: hd_dep_count@1 = 4 OR hd_vehicle_count@2 = 3, projection=[hd_demo_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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 OR hd_vehicle_count@4 = 3, 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 OR hd_vehicle_count_null_count@6 != row_count@3 AND hd_vehicle_count_min@4 <= 3 AND 3 <= hd_vehicle_count_max@5, required_guarantees=[] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] + │ FilterExec: hd_dep_count@1 = 4 OR hd_vehicle_count@2 = 3, projection=[hd_demo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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 OR hd_vehicle_count@4 = 3, 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 OR hd_vehicle_count_null_count@6 != row_count@3 AND hd_vehicle_count_min@4 <= 3 AND 3 <= hd_vehicle_count_max@5, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_city@1 = Fairview OR s_city@1 = Midway, projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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_city], file_type=parquet, predicate=s_city@22 = Fairview OR s_city@22 = Midway, pruning_predicate=s_city_null_count@2 != row_count@3 AND s_city_min@0 <= Fairview AND Fairview <= s_city_max@1 OR s_city_null_count@2 != row_count@3 AND s_city_min@0 <= Midway AND Midway <= s_city_max@1, required_guarantees=[s_city in (Fairview, Midway)] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_city@1 = Fairview OR s_city@1 = Midway, projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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_city], file_type=parquet, predicate=s_city@22 = Fairview OR s_city@22 = Midway, pruning_predicate=s_city_null_count@2 != row_count@3 AND s_city_min@0 <= Fairview AND Fairview <= s_city_max@1 OR s_city_null_count@2 != row_count@3 AND s_city_min@0 <= Midway AND Midway <= s_city_max@1, required_guarantees=[s_city in (Fairview, Midway)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_dow@2 = 6 OR d_dow@2 = 0) AND (d_year@1 = 1999 OR d_year@1 = 2000 OR d_year@1 = 2001), projection=[d_date_sk@0] - │ 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_year, d_dow], file_type=parquet, predicate=(d_dow@7 = 6 OR d_dow@7 = 0) AND (d_year@6 = 1999 OR d_year@6 = 2000 OR d_year@6 = 2001), pruning_predicate=(d_dow_null_count@2 != row_count@3 AND d_dow_min@0 <= 6 AND 6 <= d_dow_max@1 OR d_dow_null_count@2 != row_count@3 AND d_dow_min@0 <= 0 AND 0 <= d_dow_max@1) AND (d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5 OR d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5 OR d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5), required_guarantees=[d_dow in (0, 6), d_year in (1999, 2000, 2001)] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_dow@2 = 6 OR d_dow@2 = 0) AND (d_year@1 = 1999 OR d_year@1 = 2000 OR d_year@1 = 2001), projection=[d_date_sk@0] + │ 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_year, d_dow], file_type=parquet, predicate=(d_dow@7 = 6 OR d_dow@7 = 0) AND (d_year@6 = 1999 OR d_year@6 = 2000 OR d_year@6 = 2001), pruning_predicate=(d_dow_null_count@2 != row_count@3 AND d_dow_min@0 <= 6 AND 6 <= d_dow_max@1 OR d_dow_null_count@2 != row_count@3 AND d_dow_min@0 <= 0 AND 0 <= d_dow_max@1) AND (d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5 OR d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5 OR d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5), required_guarantees=[d_dow in (0, 6), d_year in (1999, 2000, 2001)] └────────────────────────────────────────────────── "); Ok(()) @@ -4924,12 +4928,12 @@ mod tests { │ SortExec: TopK(fetch=100), expr=[sum_sales@7 - avg_monthly_sales@6 ASC NULLS LAST, i_category@0 ASC NULLS LAST, i_brand@1 ASC NULLS LAST, s_store_name@2 ASC NULLS LAST, s_company_name@3 ASC NULLS LAST, d_moy@5 ASC NULLS LAST, avg_monthly_sales@6 ASC NULLS LAST, sum_sales@7 ASC NULLS LAST, psum@8 ASC NULLS LAST, nsum@9 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[i_category@0 as i_category, i_brand@1 as i_brand, s_store_name@2 as s_store_name, s_company_name@3 as s_company_name, d_year@4 as d_year, d_moy@5 as d_moy, avg_monthly_sales@7 as avg_monthly_sales, sum_sales@6 as sum_sales, sum_sales@8 as psum, sum_sales@9 as nsum] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_category@0, i_category@0), (i_brand@1, i_brand@1), (s_store_name@2, s_store_name@2), (s_company_name@3, s_company_name@3), (CAST(v1.rn AS Decimal128(21, 0))@10, v1_lead.rn - Decimal128(Some(1),20,0)@6)], projection=[i_category@0, i_brand@1, s_store_name@2, s_company_name@3, d_year@4, d_moy@5, sum_sales@6, avg_monthly_sales@7, sum_sales@9, sum_sales@15] - │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec │ ProjectionExec: expr=[i_category@0 as i_category, i_brand@1 as i_brand, s_store_name@2 as s_store_name, s_company_name@3 as s_company_name, d_year@4 as d_year, d_moy@5 as d_moy, sum_sales@6 as sum_sales, avg_monthly_sales@7 as avg_monthly_sales, rn@8 as rn, sum_sales@9 as sum_sales, CAST(rn@8 AS Decimal128(21, 0)) as CAST(v1.rn AS Decimal128(21, 0))] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_category@0, i_category@0), (i_brand@1, i_brand@1), (s_store_name@2, s_store_name@2), (s_company_name@3, s_company_name@3), (CAST(v1.rn AS Decimal128(21, 0))@9, v1_lag.rn + Decimal128(Some(1),20,0)@6)], projection=[i_category@0, i_brand@1, s_store_name@2, s_company_name@3, d_year@4, d_moy@5, sum_sales@6, avg_monthly_sales@7, rn@8, sum_sales@14] - │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec │ ProjectionExec: expr=[i_category@0 as i_category, i_brand@1 as i_brand, s_store_name@2 as s_store_name, s_company_name@3 as s_company_name, d_year@4 as d_year, d_moy@5 as d_moy, sum(store_sales.ss_sales_price)@6 as sum_sales, avg(sum(store_sales.ss_sales_price)) PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name, date_dim.d_year] ORDER BY [date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 as avg_monthly_sales, rank() PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@8 as rn, CAST(rank() PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@8 AS Decimal128(21, 0)) as CAST(v1.rn AS Decimal128(21, 0))] │ FilterExec: d_year@4 = 1999 AND avg(sum(store_sales.ss_sales_price)) PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name, date_dim.d_year] ORDER BY [date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 > Some(0),19,6 AND CASE WHEN avg(sum(store_sales.ss_sales_price)) PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name, date_dim.d_year] ORDER BY [date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 > Some(0),19,6 THEN abs(sum(store_sales.ss_sales_price)@6 - avg(sum(store_sales.ss_sales_price)) PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name, date_dim.d_year] ORDER BY [date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7) / avg(sum(store_sales.ss_sales_price)) PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name, date_dim.d_year] ORDER BY [date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 END > Some(1000000000),30,10 │ BoundedWindowAggExec: wdw=[rank() PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [item.i_category, item.i_brand, store.s_store_name, store.s_company_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] @@ -4957,35 +4961,35 @@ mod tests { │ AggregateExec: mode=Partial, gby=[i_category@1 as i_category, i_brand@0 as i_brand, s_store_name@5 as s_store_name, s_company_name@6 as s_company_name, d_year@3 as d_year, d_moy@4 as d_moy], aggr=[sum(store_sales.ss_sales_price)] │ ProjectionExec: expr=[i_brand@2 as i_brand, i_category@3 as i_category, ss_sales_price@4 as ss_sales_price, d_year@5 as d_year, d_moy@6 as d_moy, s_store_name@0 as s_store_name, s_company_name@1 as s_company_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@2)], projection=[s_store_name@1, s_company_name@2, i_brand@4, i_category@5, ss_sales_price@7, d_year@8, d_moy@9] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[i_brand@2 as i_brand, i_category@3 as i_category, ss_store_sk@4 as ss_store_sk, ss_sales_price@5 as ss_sales_price, d_year@0 as d_year, d_moy@1 as d_moy] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ss_sold_date_sk@2)], projection=[d_year@1, d_moy@2, i_brand@4, i_category@5, ss_store_sk@7, ss_sales_price@8] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand@1, i_category@2, ss_sold_date_sk@3, ss_store_sk@5, ss_sales_price@6] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_company_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_company_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 1999 OR d_year@1 = 1998 AND d_moy@2 = 12 OR d_year@1 = 2000 AND d_moy@2 = 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_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1999 OR d_year@6 = 1998 AND d_moy@8 = 12 OR d_year@6 = 2000 AND d_moy@8 = 1, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR 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 <= 12 AND 12 <= d_moy_max@5 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 1 AND 1 <= d_moy_max@5, required_guarantees=[d_year in (1998, 1999, 2000)] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 1999 OR d_year@1 = 1998 AND d_moy@2 = 12 OR d_year@1 = 2000 AND d_moy@2 = 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_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1999 OR d_year@6 = 1998 AND d_moy@8 = 12 OR d_year@6 = 2000 AND d_moy@8 = 1, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR 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 <= 12 AND 12 <= d_moy_max@5 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 1 AND 1 <= d_moy_max@5, required_guarantees=[d_year in (1998, 1999, 2000)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, s_store_name@2, s_company_name@3], 3), input_partitions=3 @@ -4997,35 +5001,35 @@ mod tests { │ AggregateExec: mode=Partial, gby=[i_category@1 as i_category, i_brand@0 as i_brand, s_store_name@5 as s_store_name, s_company_name@6 as s_company_name, d_year@3 as d_year, d_moy@4 as d_moy], aggr=[sum(store_sales.ss_sales_price)] │ ProjectionExec: expr=[i_brand@2 as i_brand, i_category@3 as i_category, ss_sales_price@4 as ss_sales_price, d_year@5 as d_year, d_moy@6 as d_moy, s_store_name@0 as s_store_name, s_company_name@1 as s_company_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@2)], projection=[s_store_name@1, s_company_name@2, i_brand@4, i_category@5, ss_sales_price@7, d_year@8, d_moy@9] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[i_brand@2 as i_brand, i_category@3 as i_category, ss_store_sk@4 as ss_store_sk, ss_sales_price@5 as ss_sales_price, d_year@0 as d_year, d_moy@1 as d_moy] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ss_sold_date_sk@2)], projection=[d_year@1, d_moy@2, i_brand@4, i_category@5, ss_store_sk@7, ss_sales_price@8] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand@1, i_category@2, ss_sold_date_sk@3, ss_store_sk@5, ss_sales_price@6] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_company_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_company_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 1999 OR d_year@1 = 1998 AND d_moy@2 = 12 OR d_year@1 = 2000 AND d_moy@2 = 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_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1999 OR d_year@6 = 1998 AND d_moy@8 = 12 OR d_year@6 = 2000 AND d_moy@8 = 1, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR 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 <= 12 AND 12 <= d_moy_max@5 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 1 AND 1 <= d_moy_max@5, required_guarantees=[d_year in (1998, 1999, 2000)] + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 1999 OR d_year@1 = 1998 AND d_moy@2 = 12 OR d_year@1 = 2000 AND d_moy@2 = 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_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1999 OR d_year@6 = 1998 AND d_moy@8 = 12 OR d_year@6 = 2000 AND d_moy@8 = 1, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR 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 <= 12 AND 12 <= d_moy_max@5 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 1 AND 1 <= d_moy_max@5, required_guarantees=[d_year in (1998, 1999, 2000)] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 15 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, s_store_name@2, s_company_name@3], 3), input_partitions=3 @@ -5037,35 +5041,35 @@ mod tests { │ AggregateExec: mode=Partial, gby=[i_category@1 as i_category, i_brand@0 as i_brand, s_store_name@5 as s_store_name, s_company_name@6 as s_company_name, d_year@3 as d_year, d_moy@4 as d_moy], aggr=[sum(store_sales.ss_sales_price)] │ ProjectionExec: expr=[i_brand@2 as i_brand, i_category@3 as i_category, ss_sales_price@4 as ss_sales_price, d_year@5 as d_year, d_moy@6 as d_moy, s_store_name@0 as s_store_name, s_company_name@1 as s_company_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@2)], projection=[s_store_name@1, s_company_name@2, i_brand@4, i_category@5, ss_sales_price@7, d_year@8, d_moy@9] - │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[i_brand@2 as i_brand, i_category@3 as i_category, ss_store_sk@4 as ss_store_sk, ss_sales_price@5 as ss_sales_price, d_year@0 as d_year, d_moy@1 as d_moy] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ss_sold_date_sk@2)], projection=[d_year@1, d_moy@2, i_brand@4, i_category@5, ss_store_sk@7, ss_sales_price@8] - │ CoalescePartitionsExec - │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand@1, i_category@2, ss_sold_date_sk@3, ss_store_sk@5, ss_sales_price@6] - │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_company_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_company_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 1999 OR d_year@1 = 1998 AND d_moy@2 = 12 OR d_year@1 = 2000 AND d_moy@2 = 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_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1999 OR d_year@6 = 1998 AND d_moy@8 = 12 OR d_year@6 = 2000 AND d_moy@8 = 1, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR 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 <= 12 AND 12 <= d_moy_max@5 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 1 AND 1 <= d_moy_max@5, required_guarantees=[d_year in (1998, 1999, 2000)] + ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 1999 OR d_year@1 = 1998 AND d_moy@2 = 12 OR d_year@1 = 2000 AND d_moy@2 = 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_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1999 OR d_year@6 = 1998 AND d_moy@8 = 12 OR d_year@6 = 2000 AND d_moy@8 = 1, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR 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 <= 12 AND 12 <= d_moy_max@5 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 1 AND 1 <= d_moy_max@5, required_guarantees=[d_year in (1998, 1999, 2000)] └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet + ┌───── Stage 13 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet └────────────────────────────────────────────────── "#); Ok(()) @@ -5082,48 +5086,48 @@ mod tests { ┌───── 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 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@2)], filter=(ca_state@1 = CO OR ca_state@1 = OH OR ca_state@1 = TX) AND ss_net_profit@0 >= Some(0),6,2 AND ss_net_profit@0 <= Some(200000),6,2 OR (ca_state@1 = OR OR ca_state@1 = MN OR ca_state@1 = KY) AND ss_net_profit@0 >= Some(15000),6,2 AND ss_net_profit@0 <= Some(300000),6,2 OR (ca_state@1 = VA OR ca_state@1 = CA OR ca_state@1 = MS) AND ss_net_profit@0 >= Some(5000),6,2 AND CAST(ss_net_profit@0 AS Decimal128(22, 2)) <= Some(2500000),22,2, projection=[ss_sold_date_sk@0, ss_quantity@2] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_state@1 as ca_state, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ FilterExec: (ca_state@1 = CO OR ca_state@1 = OH OR ca_state@1 = TX OR ca_state@1 = OR OR ca_state@1 = MN OR ca_state@1 = KY OR ca_state@1 = VA OR ca_state@1 = CA OR ca_state@1 = MS) AND ca_state@1 IN (SET) ([CO, OH, TX, OR, MN, KY, VA, CA, MS]) AND ca_country@2 = United States, projection=[ca_address_sk@0, ca_state@1] - │ 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/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state, ca_country], file_type=parquet, predicate=(ca_state@8 = CO OR ca_state@8 = OH OR ca_state@8 = TX OR ca_state@8 = OR OR ca_state@8 = MN OR ca_state@8 = KY OR ca_state@8 = VA OR ca_state@8 = CA OR ca_state@8 = MS) AND ca_state@8 IN (SET) ([CO, OH, TX, OR, MN, KY, VA, CA, MS]) AND ca_country@10 = United States, pruning_predicate=(ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= CO AND CO <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OH AND OH <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= TX AND TX <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OR AND OR <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= MN AND MN <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= KY AND KY <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= VA AND VA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= CA AND CA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= MS AND MS <= ca_state_max@1) AND (ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= CO AND CO <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OH AND OH <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= TX AND TX <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OR AND OR <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= MN AND MN <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= KY AND KY <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= VA AND VA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= CA AND CA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= MS AND MS <= ca_state_max@1) AND ca_country_null_count@6 != row_count@3 AND ca_country_min@4 <= United States AND United States <= ca_country_max@5, required_guarantees=[ca_country in (United States), ca_state in (CA, CO, KY, MN, MS, OH, OR, TX, VA)] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_cdemo_sk@1, CAST(customer_demographics.cd_demo_sk AS Float64)@3)], filter=cd_marital_status@1 = M AND cd_education_status@2 = 4 yr Degree AND ss_sales_price@0 >= Some(10000),5,2 AND ss_sales_price@0 <= Some(15000),5,2 OR cd_marital_status@1 = D AND cd_education_status@2 = 2 yr Degree AND ss_sales_price@0 >= Some(5000),5,2 AND ss_sales_price@0 <= Some(10000),5,2 OR cd_marital_status@1 = S AND cd_education_status@2 = College AND ss_sales_price@0 >= Some(15000),5,2 AND ss_sales_price@0 <= Some(20000),5,2, projection=[ss_sold_date_sk@0, ss_addr_sk@2, ss_quantity@3, ss_net_profit@5] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] - │ FilterExec: cd_marital_status@1 = M AND cd_education_status@2 = 4 yr Degree OR cd_marital_status@1 = D AND cd_education_status@2 = 2 yr Degree OR cd_marital_status@1 = S AND cd_education_status@2 = College - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_marital_status@2 = M AND cd_education_status@3 = 4 yr Degree OR cd_marital_status@2 = D AND cd_education_status@3 = 2 yr Degree OR cd_marital_status@2 = S AND cd_education_status@3 = College, pruning_predicate=cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= M AND M <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= 4 yr Degree AND 4 yr Degree <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= D AND D <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= 2 yr Degree AND 2 yr Degree <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= S AND S <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= College AND College <= cd_education_status_max@5, required_guarantees=[cd_education_status in (2 yr Degree, 4 yr Degree, College), cd_marital_status in (D, M, S)] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@3)], projection=[ss_sold_date_sk@2, ss_cdemo_sk@3, ss_addr_sk@4, ss_quantity@6, ss_sales_price@7, ss_net_profit@8] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ FilterExec: (ss_net_profit@6 >= Some(0),6,2 AND ss_net_profit@6 <= Some(200000),6,2 OR ss_net_profit@6 >= Some(15000),6,2 AND ss_net_profit@6 <= Some(300000),6,2 OR ss_net_profit@6 >= Some(5000),6,2 AND CAST(ss_net_profit@6 AS Decimal128(22, 2)) <= Some(2500000),22,2) AND (ss_sales_price@5 >= Some(10000),5,2 AND ss_sales_price@5 <= Some(15000),5,2 OR ss_sales_price@5 >= Some(5000),5,2 AND ss_sales_price@5 <= Some(10000),5,2 OR ss_sales_price@5 >= Some(15000),5,2 AND ss_sales_price@5 <= Some(20000),5,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_cdemo_sk, ss_addr_sk, ss_store_sk, ss_quantity, ss_sales_price, ss_net_profit], file_type=parquet, predicate=(ss_net_profit@22 >= Some(0),6,2 AND ss_net_profit@22 <= Some(200000),6,2 OR ss_net_profit@22 >= Some(15000),6,2 AND ss_net_profit@22 <= Some(300000),6,2 OR ss_net_profit@22 >= Some(5000),6,2 AND CAST(ss_net_profit@22 AS Decimal128(22, 2)) <= Some(2500000),22,2) AND (ss_sales_price@13 >= Some(10000),5,2 AND ss_sales_price@13 <= Some(15000),5,2 OR ss_sales_price@13 >= Some(5000),5,2 AND ss_sales_price@13 <= Some(10000),5,2 OR ss_sales_price@13 >= Some(15000),5,2 AND ss_sales_price@13 <= Some(20000),5,2) AND DynamicFilter [ empty ], pruning_predicate=(ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_max@0 >= Some(0),6,2 AND ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_min@3 <= Some(200000),6,2 OR ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_max@0 >= Some(15000),6,2 AND ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_min@3 <= Some(300000),6,2 OR ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_max@0 >= Some(5000),6,2 AND ss_net_profit_null_count@1 != row_count@2 AND CAST(ss_net_profit_min@3 AS Decimal128(22, 2)) <= Some(2500000),22,2) AND (ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_max@4 >= Some(10000),5,2 AND ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_min@6 <= Some(15000),5,2 OR ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_max@4 >= Some(5000),5,2 AND ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_min@6 <= Some(10000),5,2 OR ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_max@4 >= Some(15000),5,2 AND ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_min@6 <= Some(20000),5,2), required_guarantees=[] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@2)], filter=(ca_state@1 = CO OR ca_state@1 = OH OR ca_state@1 = TX) AND ss_net_profit@0 >= Some(0),6,2 AND ss_net_profit@0 <= Some(200000),6,2 OR (ca_state@1 = OR OR ca_state@1 = MN OR ca_state@1 = KY) AND ss_net_profit@0 >= Some(15000),6,2 AND ss_net_profit@0 <= Some(300000),6,2 OR (ca_state@1 = VA OR ca_state@1 = CA OR ca_state@1 = MS) AND ss_net_profit@0 >= Some(5000),6,2 AND CAST(ss_net_profit@0 AS Decimal128(22, 2)) <= Some(2500000),22,2, projection=[ss_sold_date_sk@0, ss_quantity@2] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_state@1 as ca_state, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] + │ FilterExec: (ca_state@1 = CO OR ca_state@1 = OH OR ca_state@1 = TX OR ca_state@1 = OR OR ca_state@1 = MN OR ca_state@1 = KY OR ca_state@1 = VA OR ca_state@1 = CA OR ca_state@1 = MS) AND ca_state@1 IN (SET) ([CO, OH, TX, OR, MN, KY, VA, CA, MS]) AND ca_country@2 = United States, projection=[ca_address_sk@0, ca_state@1] + │ 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/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state, ca_country], file_type=parquet, predicate=(ca_state@8 = CO OR ca_state@8 = OH OR ca_state@8 = TX OR ca_state@8 = OR OR ca_state@8 = MN OR ca_state@8 = KY OR ca_state@8 = VA OR ca_state@8 = CA OR ca_state@8 = MS) AND ca_state@8 IN (SET) ([CO, OH, TX, OR, MN, KY, VA, CA, MS]) AND ca_country@10 = United States, pruning_predicate=(ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= CO AND CO <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OH AND OH <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= TX AND TX <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OR AND OR <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= MN AND MN <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= KY AND KY <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= VA AND VA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= CA AND CA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= MS AND MS <= ca_state_max@1) AND (ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= CO AND CO <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OH AND OH <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= TX AND TX <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OR AND OR <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= MN AND MN <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= KY AND KY <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= VA AND VA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= CA AND CA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= MS AND MS <= ca_state_max@1) AND ca_country_null_count@6 != row_count@3 AND ca_country_min@4 <= United States AND United States <= ca_country_max@5, required_guarantees=[ca_country in (United States), ca_state in (CA, CO, KY, MN, MS, OH, OR, TX, VA)] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_cdemo_sk@1, CAST(customer_demographics.cd_demo_sk AS Float64)@3)], filter=cd_marital_status@1 = M AND cd_education_status@2 = 4 yr Degree AND ss_sales_price@0 >= Some(10000),5,2 AND ss_sales_price@0 <= Some(15000),5,2 OR cd_marital_status@1 = D AND cd_education_status@2 = 2 yr Degree AND ss_sales_price@0 >= Some(5000),5,2 AND ss_sales_price@0 <= Some(10000),5,2 OR cd_marital_status@1 = S AND cd_education_status@2 = College AND ss_sales_price@0 >= Some(15000),5,2 AND ss_sales_price@0 <= Some(20000),5,2, projection=[ss_sold_date_sk@0, ss_addr_sk@2, ss_quantity@3, ss_net_profit@5] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] + │ FilterExec: cd_marital_status@1 = M AND cd_education_status@2 = 4 yr Degree OR cd_marital_status@1 = D AND cd_education_status@2 = 2 yr Degree OR cd_marital_status@1 = S AND cd_education_status@2 = College + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_marital_status@2 = M AND cd_education_status@3 = 4 yr Degree OR cd_marital_status@2 = D AND cd_education_status@3 = 2 yr Degree OR cd_marital_status@2 = S AND cd_education_status@3 = College, pruning_predicate=cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= M AND M <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= 4 yr Degree AND 4 yr Degree <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= D AND D <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= 2 yr Degree AND 2 yr Degree <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= S AND S <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= College AND College <= cd_education_status_max@5, required_guarantees=[cd_education_status in (2 yr Degree, 4 yr Degree, College), cd_marital_status in (D, M, S)] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@3)], projection=[ss_sold_date_sk@2, ss_cdemo_sk@3, ss_addr_sk@4, ss_quantity@6, ss_sales_price@7, ss_net_profit@8] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ FilterExec: (ss_net_profit@6 >= Some(0),6,2 AND ss_net_profit@6 <= Some(200000),6,2 OR ss_net_profit@6 >= Some(15000),6,2 AND ss_net_profit@6 <= Some(300000),6,2 OR ss_net_profit@6 >= Some(5000),6,2 AND CAST(ss_net_profit@6 AS Decimal128(22, 2)) <= Some(2500000),22,2) AND (ss_sales_price@5 >= Some(10000),5,2 AND ss_sales_price@5 <= Some(15000),5,2 OR ss_sales_price@5 >= Some(5000),5,2 AND ss_sales_price@5 <= Some(10000),5,2 OR ss_sales_price@5 >= Some(15000),5,2 AND ss_sales_price@5 <= Some(20000),5,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_cdemo_sk, ss_addr_sk, ss_store_sk, ss_quantity, ss_sales_price, ss_net_profit], file_type=parquet, predicate=(ss_net_profit@22 >= Some(0),6,2 AND ss_net_profit@22 <= Some(200000),6,2 OR ss_net_profit@22 >= Some(15000),6,2 AND ss_net_profit@22 <= Some(300000),6,2 OR ss_net_profit@22 >= Some(5000),6,2 AND CAST(ss_net_profit@22 AS Decimal128(22, 2)) <= Some(2500000),22,2) AND (ss_sales_price@13 >= Some(10000),5,2 AND ss_sales_price@13 <= Some(15000),5,2 OR ss_sales_price@13 >= Some(5000),5,2 AND ss_sales_price@13 <= Some(10000),5,2 OR ss_sales_price@13 >= Some(15000),5,2 AND ss_sales_price@13 <= Some(20000),5,2) AND DynamicFilter [ empty ], pruning_predicate=(ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_max@0 >= Some(0),6,2 AND ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_min@3 <= Some(200000),6,2 OR ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_max@0 >= Some(15000),6,2 AND ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_min@3 <= Some(300000),6,2 OR ss_net_profit_null_count@1 != row_count@2 AND ss_net_profit_max@0 >= Some(5000),6,2 AND ss_net_profit_null_count@1 != row_count@2 AND CAST(ss_net_profit_min@3 AS Decimal128(22, 2)) <= Some(2500000),22,2) AND (ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_max@4 >= Some(10000),5,2 AND ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_min@6 <= Some(15000),5,2 OR ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_max@4 >= Some(5000),5,2 AND ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_min@6 <= Some(10000),5,2 OR ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_max@4 >= Some(15000),5,2 AND ss_sales_price_null_count@5 != row_count@2 AND ss_sales_price_min@6 <= Some(20000),5,2), required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── "); Ok(()) @@ -5178,21 +5182,21 @@ mod tests { │ AggregateExec: mode=Partial, gby=[ws_item_sk@2 as ws_item_sk], aggr=[sum(coalesce(wr.wr_return_quantity,Int64(0))), sum(coalesce(ws.ws_quantity,Int64(0))), sum(coalesce(wr.wr_return_amt,Int64(0))), sum(coalesce(ws.ws_net_paid,Int64(0)))] │ ProjectionExec: expr=[CAST(wr_return_amt@4 AS Decimal128(22, 2)) as __common_expr_1, CAST(ws_net_paid@2 AS Decimal128(22, 2)) as __common_expr_2, ws_item_sk@0 as ws_item_sk, ws_quantity@1 as ws_quantity, wr_return_quantity@3 as wr_return_quantity] │ 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_net_paid@5, wr_return_quantity@6, wr_return_amt@7] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, input_tasks=2 │ FilterExec: wr_return_amt@5 > Some(1000000),7,2 │ ProjectionExec: expr=[ws_sold_date_sk@2 as ws_sold_date_sk, ws_item_sk@3 as ws_item_sk, ws_quantity@4 as ws_quantity, ws_net_paid@5 as ws_net_paid, wr_return_quantity@0 as wr_return_quantity, wr_return_amt@1 as wr_return_amt] │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(wr_order_number@1, ws_order_number@2), (wr_item_sk@0, ws_item_sk@1)], projection=[wr_return_quantity@2, wr_return_amt@3, ws_sold_date_sk@4, ws_item_sk@5, ws_quantity@7, ws_net_paid@8] │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=2 │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ 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 AND d_moy@2 = 12, projection=[d_date_sk@0] - │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 12, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5, required_guarantees=[d_moy in (12), d_year in (2001)] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ 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 AND d_moy@2 = 12, projection=[d_date_sk@0] + │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 12, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5, required_guarantees=[d_moy in (12), d_year in (2001)] └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] │ RepartitionExec: partitioning=Hash([wr_order_number@1, wr_item_sk@0], 6), input_partitions=2 @@ -5216,21 +5220,21 @@ mod tests { │ AggregateExec: mode=Partial, gby=[cs_item_sk@2 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)))] │ ProjectionExec: expr=[CAST(cr_return_amount@4 AS Decimal128(22, 2)) as __common_expr_3, CAST(cs_net_paid@2 AS Decimal128(22, 2)) as __common_expr_4, cs_item_sk@0 as cs_item_sk, cs_quantity@1 as cs_quantity, cr_return_quantity@3 as cr_return_quantity] │ 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_net_paid@5, cr_return_quantity@6, cr_return_amount@7] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ FilterExec: cr_return_amount@5 > Some(1000000),7,2 │ ProjectionExec: expr=[cs_sold_date_sk@2 as cs_sold_date_sk, cs_item_sk@3 as cs_item_sk, cs_quantity@4 as cs_quantity, cs_net_paid@5 as cs_net_paid, cr_return_quantity@0 as cr_return_quantity, cr_return_amount@1 as cr_return_amount] │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(cr_order_number@1, cs_order_number@2), (cr_item_sk@0, cs_item_sk@1)], projection=[cr_return_quantity@2, cr_return_amount@3, cs_sold_date_sk@4, cs_item_sk@5, cs_quantity@7, cs_net_paid@8] │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 AND d_moy@2 = 12, projection=[d_date_sk@0] - │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 12, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5, required_guarantees=[d_moy in (12), d_year in (2001)] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 AND d_moy@2 = 12, projection=[d_date_sk@0] + │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 12, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5, required_guarantees=[d_moy in (12), d_year in (2001)] └────────────────────────────────────────────────── ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ RepartitionExec: partitioning=Hash([cr_order_number@1, cr_item_sk@0], 9), input_partitions=2 @@ -5254,21 +5258,21 @@ mod tests { │ AggregateExec: mode=Partial, gby=[ss_item_sk@2 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)))] │ ProjectionExec: expr=[CAST(sr_return_amt@4 AS Decimal128(22, 2)) as __common_expr_5, CAST(ss_net_paid@2 AS Decimal128(22, 2)) as __common_expr_6, ss_item_sk@0 as ss_item_sk, ss_quantity@1 as ss_quantity, sr_return_quantity@3 as sr_return_quantity] │ 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_net_paid@5, sr_return_quantity@6, sr_return_amt@7] - │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ FilterExec: sr_return_amt@5 > Some(1000000),7,2 │ ProjectionExec: expr=[ss_sold_date_sk@2 as ss_sold_date_sk, ss_item_sk@3 as ss_item_sk, ss_quantity@4 as ss_quantity, ss_net_paid@5 as ss_net_paid, sr_return_quantity@0 as sr_return_quantity, sr_return_amt@1 as sr_return_amt] │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(sr_ticket_number@1, ss_ticket_number@2), (sr_item_sk@0, ss_item_sk@1)], projection=[sr_return_quantity@2, sr_return_amt@3, ss_sold_date_sk@4, ss_item_sk@5, ss_quantity@7, ss_net_paid@8] │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 AND d_moy@2 = 12, projection=[d_date_sk@0] - │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 12, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5, required_guarantees=[d_moy in (12), d_year in (2001)] + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 AND d_moy@2 = 12, projection=[d_date_sk@0] + │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 12, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2001 AND 2001 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 12 AND 12 <= d_moy_max@5, required_guarantees=[d_moy in (12), d_year in (2001)] └────────────────────────────────────────────────── ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ RepartitionExec: partitioning=Hash([sr_ticket_number@1, sr_item_sk@0], 9), input_partitions=2 @@ -5303,39 +5307,39 @@ mod tests { │ AggregateExec: mode=Partial, gby=[s_store_name@1 as s_store_name, s_company_id@2 as s_company_id, s_street_number@3 as s_street_number, s_street_name@4 as s_street_name, s_street_type@5 as s_street_type, s_suite_number@6 as s_suite_number, s_city@7 as s_city, s_county@8 as s_county, s_state@9 as s_state, s_zip@10 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)] │ ProjectionExec: expr=[sr_returned_date_sk@1 - ss_sold_date_sk@0 as __common_expr_1, s_store_name@2 as s_store_name, s_company_id@3 as s_company_id, s_street_number@4 as s_street_number, s_street_name@5 as s_street_name, s_street_type@6 as s_street_type, s_suite_number@7 as s_suite_number, s_city@8 as s_city, s_county@9 as s_county, s_state@10 as s_state, s_zip@11 as s_zip] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(sr_returned_date_sk@1, CAST(d2.d_date_sk AS Float64)@1)], projection=[ss_sold_date_sk@0, sr_returned_date_sk@1, s_store_name@2, s_company_id@3, s_street_number@4, s_street_name@5, s_street_type@6, s_suite_number@7, s_city@8, s_county@9, s_state@10, s_zip@11] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d2.d_date_sk AS Float64)] │ FilterExec: d_year@1 = 2001 AND d_moy@2 = 8, 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, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 = 8, 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 <= 8 AND 8 <= d_moy_max@5, required_guarantees=[d_moy in (8), 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 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_sold_date_sk@0, CAST(d1.d_date_sk AS Float64)@1)], projection=[ss_sold_date_sk@0, sr_returned_date_sk@1, s_store_name@2, s_company_id@3, s_street_number@4, s_street_name@5, s_street_type@6, s_suite_number@7, s_city@8, s_county@9, s_state@10, s_zip@11] - │ 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(d1.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], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[ss_sold_date_sk@10 as ss_sold_date_sk, sr_returned_date_sk@11 as sr_returned_date_sk, 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] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@11, ss_store_sk@1)], projection=[s_store_name@1, s_company_id@2, s_street_number@3, s_street_name@4, s_street_type@5, s_suite_number@6, s_city@7, s_county@8, s_state@9, s_zip@10, ss_sold_date_sk@12, sr_returned_date_sk@14] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[ss_sold_date_sk@1 as ss_sold_date_sk, ss_store_sk@2 as ss_store_sk, sr_returned_date_sk@0 as sr_returned_date_sk] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_ticket_number@3, ss_ticket_number@4), (sr_item_sk@1, ss_item_sk@1), (sr_customer_sk@2, ss_customer_sk@2)], projection=[sr_returned_date_sk@0, ss_sold_date_sk@4, ss_store_sk@7] - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_company_id, s_street_number, s_street_name, s_street_type, s_suite_number, s_city, s_county, s_state, s_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_sold_date_sk@0, CAST(d1.d_date_sk AS Float64)@1)], projection=[ss_sold_date_sk@0, sr_returned_date_sk@1, s_store_name@2, s_company_id@3, s_street_number@4, s_street_name@5, s_street_type@6, s_suite_number@7, s_city@8, s_county@9, s_state@10, s_zip@11] + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ss_sold_date_sk@10 as ss_sold_date_sk, sr_returned_date_sk@11 as sr_returned_date_sk, 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] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@11, ss_store_sk@1)], projection=[s_store_name@1, s_company_id@2, s_street_number@3, s_street_name@4, s_street_type@5, s_suite_number@6, s_city@7, s_county@8, s_state@9, s_zip@10, ss_sold_date_sk@12, sr_returned_date_sk@14] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ ProjectionExec: expr=[ss_sold_date_sk@1 as ss_sold_date_sk, ss_store_sk@2 as ss_store_sk, sr_returned_date_sk@0 as sr_returned_date_sk] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_ticket_number@3, ss_ticket_number@4), (sr_item_sk@1, ss_item_sk@1), (sr_customer_sk@2, ss_customer_sk@2)], projection=[sr_returned_date_sk@0, ss_sold_date_sk@4, ss_store_sk@7] + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_company_id, s_street_number, s_street_name, s_street_type, s_suite_number, s_city, s_county, s_state, s_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ RepartitionExec: partitioning=Hash([sr_ticket_number@3, sr_item_sk@1, sr_customer_sk@2], 9), input_partitions=2 @@ -5384,19 +5388,19 @@ mod tests { │ AggregateExec: mode=Partial, gby=[ws_item_sk@0 as ws_item_sk, d_date@2 as d_date], aggr=[sum(web_sales.ws_sales_price)] │ ProjectionExec: expr=[ws_item_sk@1 as ws_item_sk, ws_sales_price@2 as ws_sales_price, d_date@0 as d_date] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ws_sold_date_sk@0)], projection=[d_date@1, ws_item_sk@4, ws_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ FilterExec: ws_item_sk@1 IS NOT NULL │ 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_sales_price], file_type=parquet, predicate=ws_item_sk@3 IS NOT NULL AND DynamicFilter [ empty ], pruning_predicate=ws_item_sk_null_count@1 != row_count@0, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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)] - │ FilterExec: d_month_seq@2 >= 1200 AND d_month_seq@2 <= 1211, projection=[d_date_sk@0, d_date@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_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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)] + │ FilterExec: d_month_seq@2 >= 1200 AND d_month_seq@2 <= 1211, projection=[d_date_sk@0, d_date@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_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ RepartitionExec: partitioning=Hash([ss_item_sk@0], 3), input_partitions=3 @@ -5408,19 +5412,19 @@ mod tests { │ AggregateExec: mode=Partial, gby=[ss_item_sk@0 as ss_item_sk, d_date@2 as d_date], aggr=[sum(store_sales.ss_sales_price)] │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_sales_price@2 as ss_sales_price, d_date@0 as d_date] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_date@1, ss_item_sk@4, ss_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ FilterExec: ss_item_sk@1 IS NOT NULL │ 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_sales_price], file_type=parquet, predicate=ss_item_sk@2 IS NOT NULL AND DynamicFilter [ empty ], pruning_predicate=ss_item_sk_null_count@1 != row_count@0, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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)] - │ FilterExec: d_month_seq@2 >= 1200 AND d_month_seq@2 <= 1211, projection=[d_date_sk@0, d_date@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_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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)] + │ FilterExec: d_month_seq@2 >= 1200 AND d_month_seq@2 <= 1211, projection=[d_date_sk@0, d_date@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_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── "#); Ok(()) @@ -5443,28 +5447,28 @@ mod tests { │ RepartitionExec: partitioning=Hash([d_year@0, i_brand@1, i_brand_id@2], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[d_year@0 as d_year, i_brand@3 as i_brand, i_brand_id@2 as i_brand_id], aggr=[sum(store_sales.ss_ext_sales_price)], ordering_mode=PartiallySorted([0]) │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@1, i_item_sk@0)], projection=[d_year@0, ss_ext_sales_price@2, i_brand_id@4, i_brand@5] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ FilterExec: i_manager_id@3 = 1, projection=[i_item_sk@0, i_brand_id@1, i_brand@2] │ 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_brand_id, i_brand, i_manager_id], file_type=parquet, predicate=i_manager_id@20 = 1 AND DynamicFilter [ empty ], pruning_predicate=i_manager_id_null_count@2 != row_count@3 AND i_manager_id_min@0 <= 1 AND 1 <= i_manager_id_max@1, required_guarantees=[i_manager_id in (1)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(dt.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_ext_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 1] => 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_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(dt.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_ext_sales_price@5] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(dt.d_date_sk AS Float64)] - │ FilterExec: d_moy@2 = 11 AND d_year@1 = 2000, projection=[d_date_sk@0, d_year@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_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 11 AND d_year@6 = 2000, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_moy in (11), d_year in (2000)] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(dt.d_date_sk AS Float64)] + │ FilterExec: d_moy@2 = 11 AND d_year@1 = 2000, projection=[d_date_sk@0, d_year@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_date_sk, d_year, d_moy], file_type=parquet, predicate=d_moy@8 = 11 AND d_year@6 = 2000, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5, required_guarantees=[d_moy in (11), d_year in (2000)] └────────────────────────────────────────────────── "); Ok(()) @@ -5492,37 +5496,37 @@ mod tests { │ RepartitionExec: partitioning=Hash([i_manufact_id@0, d_qoy@1], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[i_manufact_id@0 as i_manufact_id, d_qoy@2 as d_qoy], aggr=[sum(store_sales.ss_sales_price)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@1)], projection=[i_manufact_id@2, ss_sales_price@4, d_qoy@5] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[i_manufact_id@1 as i_manufact_id, ss_store_sk@2 as ss_store_sk, ss_sales_price@3 as ss_sales_price, d_qoy@0 as d_qoy] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@1)], projection=[d_qoy@1, i_manufact_id@3, ss_store_sk@5, ss_sales_price@6] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_manufact_id@1, ss_sold_date_sk@2, ss_store_sk@4, ss_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_qoy@1 as d_qoy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_month_seq@1 IN (SET) ([1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211]), projection=[d_date_sk@0, d_qoy@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/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, d_qoy], file_type=parquet, predicate=d_month_seq@3 IN (SET) ([1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211]), pruning_predicate=d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1200 AND 1200 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1201 AND 1201 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1202 AND 1202 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1203 AND 1203 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1204 AND 1204 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1205 AND 1205 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1206 AND 1206 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1207 AND 1207 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1208 AND 1208 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1209 AND 1209 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1210 AND 1210 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1211 AND 1211 <= d_month_seq_max@1, required_guarantees=[d_month_seq in (1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_qoy@1 as d_qoy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_month_seq@1 IN (SET) ([1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211]), projection=[d_date_sk@0, d_qoy@2] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: (i_category@3 = Books OR i_category@3 = Children OR i_category@3 = Electronics) AND i_class@2 IN (SET) ([personal, portable, reference, self-help]) AND i_brand@1 IN (SET) ([scholaramalgamalg #14, scholaramalgamalg #7, exportiunivamalg #9, scholaramalgamalg #9]) OR (i_category@3 = Women OR i_category@3 = Music OR i_category@3 = Men) AND i_class@2 IN (SET) ([accessories, classical, fragrances, pants]) AND i_brand@1 IN (SET) ([amalgimporto #1, edu packscholar #1, exportiimporto #1, importoamalg #1]), projection=[i_item_sk@0, i_manufact_id@4] │ 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_month_seq, d_qoy], file_type=parquet, predicate=d_month_seq@3 IN (SET) ([1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211]), pruning_predicate=d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1200 AND 1200 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1201 AND 1201 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1202 AND 1202 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1203 AND 1203 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1204 AND 1204 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1205 AND 1205 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1206 AND 1206 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1207 AND 1207 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1208 AND 1208 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1209 AND 1209 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1210 AND 1210 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1211 AND 1211 <= d_month_seq_max@1, required_guarantees=[d_month_seq in (1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211)] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: (i_category@3 = Books OR i_category@3 = Children OR i_category@3 = Electronics) AND i_class@2 IN (SET) ([personal, portable, reference, self-help]) AND i_brand@1 IN (SET) ([scholaramalgamalg #14, scholaramalgamalg #7, exportiunivamalg #9, scholaramalgamalg #9]) OR (i_category@3 = Women OR i_category@3 = Music OR i_category@3 = Men) AND i_class@2 IN (SET) ([accessories, classical, fragrances, pants]) AND i_brand@1 IN (SET) ([amalgimporto #1, edu packscholar #1, exportiimporto #1, importoamalg #1]), projection=[i_item_sk@0, i_manufact_id@4] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_class, i_category, i_manufact_id], file_type=parquet, predicate=(i_category@12 = Books OR i_category@12 = Children OR i_category@12 = Electronics) AND i_class@10 IN (SET) ([personal, portable, reference, self-help]) AND i_brand@8 IN (SET) ([scholaramalgamalg #14, scholaramalgamalg #7, exportiunivamalg #9, scholaramalgamalg #9]) OR (i_category@12 = Women OR i_category@12 = Music OR i_category@12 = Men) AND i_class@10 IN (SET) ([accessories, classical, fragrances, pants]) AND i_brand@8 IN (SET) ([amalgimporto #1, edu packscholar #1, exportiimporto #1, importoamalg #1]), pruning_predicate=(i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Children AND Children <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Electronics AND Electronics <= i_category_max@1) AND (i_class_null_count@6 != row_count@3 AND i_class_min@4 <= personal AND personal <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= portable AND portable <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= reference AND reference <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= self-help AND self-help <= i_class_max@5) AND (i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= scholaramalgamalg #14 AND scholaramalgamalg #14 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= scholaramalgamalg #7 AND scholaramalgamalg #7 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= exportiunivamalg #9 AND exportiunivamalg #9 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= scholaramalgamalg #9 AND scholaramalgamalg #9 <= i_brand_max@8) OR (i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Women AND Women <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Music AND Music <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Men AND Men <= i_category_max@1) AND (i_class_null_count@6 != row_count@3 AND i_class_min@4 <= accessories AND accessories <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= classical AND classical <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= fragrances AND fragrances <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= pants AND pants <= i_class_max@5) AND (i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= amalgimporto #1 AND amalgimporto #1 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= edu packscholar #1 AND edu packscholar #1 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= exportiimporto #1 AND exportiimporto #1 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= importoamalg #1 AND importoamalg #1 <= i_brand_max@8), required_guarantees=[i_brand in (amalgimporto #1, edu packscholar #1, exportiimporto #1, exportiunivamalg #9, importoamalg #1, scholaramalgamalg #14, scholaramalgamalg #7, scholaramalgamalg #9), i_class in (accessories, classical, fragrances, pants, personal, portable, reference, self-help)] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_class, i_category, i_manufact_id], file_type=parquet, predicate=(i_category@12 = Books OR i_category@12 = Children OR i_category@12 = Electronics) AND i_class@10 IN (SET) ([personal, portable, reference, self-help]) AND i_brand@8 IN (SET) ([scholaramalgamalg #14, scholaramalgamalg #7, exportiunivamalg #9, scholaramalgamalg #9]) OR (i_category@12 = Women OR i_category@12 = Music OR i_category@12 = Men) AND i_class@10 IN (SET) ([accessories, classical, fragrances, pants]) AND i_brand@8 IN (SET) ([amalgimporto #1, edu packscholar #1, exportiimporto #1, importoamalg #1]), pruning_predicate=(i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Children AND Children <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Electronics AND Electronics <= i_category_max@1) AND (i_class_null_count@6 != row_count@3 AND i_class_min@4 <= personal AND personal <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= portable AND portable <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= reference AND reference <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= self-help AND self-help <= i_class_max@5) AND (i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= scholaramalgamalg #14 AND scholaramalgamalg #14 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= scholaramalgamalg #7 AND scholaramalgamalg #7 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= exportiunivamalg #9 AND exportiunivamalg #9 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= scholaramalgamalg #9 AND scholaramalgamalg #9 <= i_brand_max@8) OR (i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Women AND Women <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Music AND Music <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Men AND Men <= i_category_max@1) AND (i_class_null_count@6 != row_count@3 AND i_class_min@4 <= accessories AND accessories <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= classical AND classical <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= fragrances AND fragrances <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= pants AND pants <= i_class_max@5) AND (i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= amalgimporto #1 AND amalgimporto #1 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= edu packscholar #1 AND edu packscholar #1 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= exportiimporto #1 AND exportiimporto #1 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= importoamalg #1 AND importoamalg #1 <= i_brand_max@8), required_guarantees=[i_brand in (amalgimporto #1, edu packscholar #1, exportiimporto #1, exportiunivamalg #9, importoamalg #1, scholaramalgamalg #14, scholaramalgamalg #7, scholaramalgamalg #9), i_class in (accessories, classical, fragrances, pants, personal, portable, reference, self-help)] └────────────────────────────────────────────────── "#); Ok(()) @@ -5544,100 +5548,113 @@ mod tests { │ AggregateExec: mode=Partial, gby=[c_customer_sk@0 as c_customer_sk], aggr=[sum(store_sales.ss_ext_sales_price)] │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, ss_ext_sales_price@1 as ss_ext_sales_price] │ NestedLoopJoinExec: join_type=Inner, filter=join_proj_push_down_2@1 <= date_dim.d_month_seq + Int64(3)@0, projection=[c_customer_sk@0, ss_ext_sales_price@1, d_month_seq@2, date_dim.d_month_seq + Int64(3)@4] - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, ss_ext_sales_price@1 as ss_ext_sales_price, d_month_seq@2 as d_month_seq, CAST(d_month_seq@2 AS Int64) as join_proj_push_down_2] - │ CoalescePartitionsExec - │ ProjectionExec: expr=[c_customer_sk@1 as c_customer_sk, ss_ext_sales_price@2 as ss_ext_sales_price, d_month_seq@3 as d_month_seq] - │ NestedLoopJoinExec: join_type=Inner, filter=join_proj_push_down_1@1 >= date_dim.d_month_seq + Int64(1)@0, projection=[date_dim.d_month_seq + Int64(1)@0, c_customer_sk@1, ss_ext_sales_price@2, d_month_seq@3] - │ CoalescePartitionsExec - │ AggregateExec: mode=FinalPartitioned, gby=[date_dim.d_month_seq + Int64(1)@0 as date_dim.d_month_seq + Int64(1)], aggr=[] - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, ss_ext_sales_price@1 as ss_ext_sales_price, d_month_seq@2 as d_month_seq, CAST(d_month_seq@2 AS Int64) as join_proj_push_down_1] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_sold_date_sk@1, CAST(date_dim.d_date_sk AS Float64)@2)], projection=[c_customer_sk@0, ss_ext_sales_price@2, d_month_seq@4] - │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_county@0, ca_county@3), (CAST(store.s_state AS Utf8View)@2, ca_state@4)], projection=[c_customer_sk@3, ss_sold_date_sk@4, ss_ext_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=2, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@1, ca_address_sk@0)], projection=[c_customer_sk@0, ss_sold_date_sk@2, ss_ext_sales_price@3, ca_county@5, ca_state@6] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=2, input_tasks=3 - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ DataSourceExec: file_groups={2 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_county, ca_state], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - │ 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(date_dim.d_date_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_month_seq], file_type=parquet + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, ss_ext_sales_price@1 as ss_ext_sales_price, d_month_seq@2 as d_month_seq, CAST(d_month_seq@2 AS Int64) as join_proj_push_down_2] + │ CoalescePartitionsExec + │ [Stage 11] => NetworkCoalesceExec: output_partitions=9, input_tasks=3 │ AggregateExec: mode=FinalPartitioned, gby=[date_dim.d_month_seq + Int64(3)@0 as date_dim.d_month_seq + Int64(3)], aggr=[] - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p0..p2] - │ RepartitionExec: partitioning=Hash([date_dim.d_month_seq + Int64(1)@0], 3), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[date_dim.d_month_seq + Int64(1)@0 as date_dim.d_month_seq + Int64(1)], aggr=[] - │ ProjectionExec: expr=[CAST(d_month_seq@0 AS Int64) + 1 as date_dim.d_month_seq + Int64(1)] - │ FilterExec: d_year@1 = 1998 AND d_moy@2 = 12, projection=[d_month_seq@0] - │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 12, 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 <= 12 AND 12 <= d_moy_max@5, required_guarantees=[d_moy in (12), d_year in (1998)] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] - │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 - │ 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_county, s_state, CAST(s_state@24 AS Utf8View) as CAST(store.s_state AS Utf8View)], file_type=parquet + ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ ProjectionExec: expr=[c_customer_sk@1 as c_customer_sk, ss_ext_sales_price@2 as ss_ext_sales_price, d_month_seq@3 as d_month_seq] + │ NestedLoopJoinExec: join_type=Inner, filter=join_proj_push_down_1@1 >= date_dim.d_month_seq + Int64(1)@0, projection=[date_dim.d_month_seq + Int64(1)@0, c_customer_sk@1, ss_ext_sales_price@2, d_month_seq@3] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=1 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, ss_ext_sales_price@1 as ss_ext_sales_price, d_month_seq@2 as d_month_seq, CAST(d_month_seq@2 AS Int64) as join_proj_push_down_1] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_sold_date_sk@1, CAST(date_dim.d_date_sk AS Float64)@2)], projection=[c_customer_sk@0, ss_ext_sales_price@2, d_month_seq@4] + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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(date_dim.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 └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] - │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(my_customers.c_customer_sk AS Float64)@2, ss_customer_sk@1)], projection=[c_customer_sk@0, c_current_addr_sk@1, ss_sold_date_sk@3, ss_ext_sales_price@5] - │ 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_date_sk, ss_customer_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, CAST(c_customer_sk@0 AS Float64) as CAST(my_customers.c_customer_sk AS Float64)] - │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk], aggr=[] - │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=4 - └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_customer_sk@0, c_current_addr_sk@1], 6), input_partitions=3 - │ AggregateExec: mode=Partial, gby=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk], aggr=[] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_sk@0, CAST(customer.c_customer_sk AS Float64)@2)], projection=[c_customer_sk@1, c_current_addr_sk@2] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 - │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk], file_type=parquet + ┌───── Stage 2 ── Tasks: t0:[p0..p2] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ AggregateExec: mode=FinalPartitioned, gby=[date_dim.d_month_seq + Int64(1)@0 as date_dim.d_month_seq + Int64(1)], aggr=[] + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p0..p2] + │ RepartitionExec: partitioning=Hash([date_dim.d_month_seq + Int64(1)@0], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[date_dim.d_month_seq + Int64(1)@0 as date_dim.d_month_seq + Int64(1)], aggr=[] + │ ProjectionExec: expr=[CAST(d_month_seq@0 AS Int64) + 1 as date_dim.d_month_seq + Int64(1)] + │ FilterExec: d_year@1 = 1998 AND d_moy@2 = 12, projection=[d_month_seq@0] + │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 12, 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 <= 12 AND 12 <= d_moy_max@5, required_guarantees=[d_moy in (12), d_year in (1998)] + └────────────────────────────────────────────────── + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_county@0, ca_county@3), (CAST(store.s_state AS Utf8View)@2, ca_state@4)], projection=[c_customer_sk@3, ss_sold_date_sk@4, ss_ext_sales_price@5] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@1, ca_address_sk@0)], projection=[c_customer_sk@0, ss_sold_date_sk@2, ss_ext_sales_price@3, ca_county@5, ca_state@6] + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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/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_county, ca_state], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_county, s_state, CAST(s_state@24 AS Utf8View) as CAST(store.s_state AS Utf8View)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] - │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, sold_date_sk@0)], projection=[customer_sk@3] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, item_sk@2)], projection=[sold_date_sk@1, customer_sk@2] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 - │ DistributedUnionExec: t0:[c0(0/2)] t1:[c0(1/2)] t2:[c1(0/2)] t3:[c1(1/2)] - │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk@0 as sold_date_sk, cs_bill_customer_sk@3 as customer_sk, cs_item_sk@15 as item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - │ 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@0 as sold_date_sk, ws_bill_customer_sk@4 as customer_sk, ws_item_sk@3 as item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p12..p23] - │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 - │ 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_moy@2 = 12 AND d_year@1 = 1998, projection=[d_date_sk@0] - │ 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_year, d_moy], file_type=parquet, predicate=d_moy@8 = 12 AND d_year@6 = 1998, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 12 AND 12 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1998 AND 1998 <= d_year_max@5, required_guarantees=[d_moy in (12), d_year in (1998)] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p12..p23] - │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 - │ FilterExec: i_category@2 = Women AND i_class@1 = maternity, projection=[i_item_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_class, i_category], file_type=parquet, predicate=i_category@12 = Women AND i_class@10 = maternity, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Women AND Women <= i_category_max@1 AND i_class_null_count@6 != row_count@3 AND i_class_min@4 <= maternity AND maternity <= i_class_max@5, required_guarantees=[i_category in (Women), i_class in (maternity)] + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(my_customers.c_customer_sk AS Float64)@2, ss_customer_sk@1)], projection=[c_customer_sk@0, c_current_addr_sk@1, ss_sold_date_sk@3, ss_ext_sales_price@5] + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_customer_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, CAST(c_customer_sk@0 AS Float64) as CAST(my_customers.c_customer_sk AS Float64)] + │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk], aggr=[] + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] + │ RepartitionExec: partitioning=Hash([c_customer_sk@0, c_current_addr_sk@1], 6), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk], aggr=[] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_sk@0, CAST(customer.c_customer_sk AS Float64)@2)], projection=[c_customer_sk@1, c_current_addr_sk@2] + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p0..p2] + ┌───── Stage 6 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, sold_date_sk@0)], projection=[customer_sk@3] + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, item_sk@2)], projection=[sold_date_sk@1, customer_sk@2] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=2 + │ DistributedUnionExec: t0:[c0(0/2)] t1:[c0(1/2)] t2:[c1(0/2)] t3:[c1(1/2)] + │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk@0 as sold_date_sk, cs_bill_customer_sk@3 as customer_sk, cs_item_sk@15 as item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ 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@0 as sold_date_sk, ws_bill_customer_sk@4 as customer_sk, ws_item_sk@3 as item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p3] t1:[p4..p7] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ 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_moy@2 = 12 AND d_year@1 = 1998, projection=[d_date_sk@0] + │ 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_year, d_moy], file_type=parquet, predicate=d_moy@8 = 12 AND d_year@6 = 1998, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 12 AND 12 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1998 AND 1998 <= d_year_max@5, required_guarantees=[d_moy in (12), d_year in (1998)] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p3] t1:[p4..p7] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ FilterExec: i_category@2 = Women AND i_class@1 = maternity, projection=[i_item_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_class, i_category], file_type=parquet, predicate=i_category@12 = Women AND i_class@10 = maternity, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Women AND Women <= i_category_max@1 AND i_class_null_count@6 != row_count@3 AND i_class_min@4 <= maternity AND maternity <= i_class_max@5, required_guarantees=[i_category in (Women), i_class in (maternity)] + └────────────────────────────────────────────────── + ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ RepartitionExec: partitioning=Hash([date_dim.d_month_seq + Int64(3)@0], 3), input_partitions=3 │ AggregateExec: mode=Partial, gby=[date_dim.d_month_seq + Int64(3)@0 as date_dim.d_month_seq + Int64(3)], aggr=[] │ ProjectionExec: expr=[CAST(d_month_seq@0 AS Int64) + 3 as date_dim.d_month_seq + Int64(3)] @@ -5667,28 +5684,28 @@ mod tests { │ RepartitionExec: partitioning=Hash([i_brand@0, i_brand_id@1], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[i_brand@2 as i_brand, i_brand_id@1 as i_brand_id], 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, i_brand_id@3, i_brand@4] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ FilterExec: i_manager_id@3 = 28, projection=[i_item_sk@0, i_brand_id@1, i_brand@2] │ 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_brand_id, i_brand, i_manager_id], file_type=parquet, predicate=i_manager_id@20 = 28 AND DynamicFilter [ empty ], pruning_predicate=i_manager_id_null_count@2 != row_count@3 AND i_manager_id_min@0 <= 28 AND 28 <= i_manager_id_max@1, required_guarantees=[i_manager_id in (28)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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_ext_sales_price@4] - │ CoalescePartitionsExec - │ [Stage 1] => 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_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ 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_ext_sales_price@4] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_moy@2 = 11 AND d_year@1 = 1999, projection=[d_date_sk@0] - │ 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_year, d_moy], file_type=parquet, predicate=d_moy@8 = 11 AND d_year@6 = 1999, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5, required_guarantees=[d_moy in (11), d_year in (1999)] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_moy@2 = 11 AND d_year@1 = 1999, projection=[d_date_sk@0] + │ 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_year, d_moy], file_type=parquet, predicate=d_moy@8 = 11 AND d_year@6 = 1999, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5, required_guarantees=[d_moy in (11), d_year in (1999)] └────────────────────────────────────────────────── "); Ok(()) @@ -5736,108 +5753,108 @@ mod tests { └────────────────────────────────────────────────── ┌───── 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 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[ss_item_sk@0, ss_ext_sales_price@2] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_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/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_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[ss_item_sk@0, ss_ext_sales_price@2] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] + │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_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/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_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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_addr_sk@4, ss_ext_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 1] => 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_addr_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ 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_addr_sk@4, ss_ext_sales_price@5] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_addr_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 AND d_moy@2 = 2, projection=[d_date_sk@0] - │ 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_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 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 AND d_moy@2 = 2, projection=[d_date_sk@0] + │ 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_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 + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_bill_addr_sk@0, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[cs_item_sk@1, cs_ext_sales_price@2] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_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/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_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_bill_addr_sk@0, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[cs_item_sk@1, cs_ext_sales_price@2] + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] + │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_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/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_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_bill_addr_sk@3, cs_item_sk@4, cs_ext_sales_price@5] - │ 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_bill_addr_sk, cs_item_sk, cs_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_bill_addr_sk@3, cs_item_sk@4, cs_ext_sales_price@5] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_addr_sk, cs_item_sk, cs_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 AND d_moy@2 = 2, projection=[d_date_sk@0] - │ 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_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 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 AND d_moy@2 = 2, projection=[d_date_sk@0] + │ 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_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 + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_bill_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[ws_item_sk@0, ws_ext_sales_price@2] - │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_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/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_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_bill_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[ws_item_sk@0, ws_ext_sales_price@2] + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] + │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_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/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_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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_bill_addr_sk@4, ws_ext_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 9] => 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_bill_addr_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ 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_bill_addr_sk@4, ws_ext_sales_price@5] + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_addr_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 AND d_moy@2 = 2, projection=[d_date_sk@0] - │ 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_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 9 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 AND d_moy@2 = 2, projection=[d_date_sk@0] + │ 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_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)] └────────────────────────────────────────────────── "); Ok(()) @@ -5851,12 +5868,12 @@ mod tests { │ SortExec: TopK(fetch=100), expr=[sum_sales@6 - avg_monthly_sales@5 ASC, i_category@0 ASC NULLS LAST, i_brand@1 ASC NULLS LAST, cc_name@2 ASC NULLS LAST, d_moy@4 ASC NULLS LAST, avg_monthly_sales@5 ASC NULLS LAST, sum_sales@6 ASC NULLS LAST, psum@7 ASC NULLS LAST, nsum@8 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[i_category@0 as i_category, i_brand@1 as i_brand, cc_name@2 as cc_name, d_year@3 as d_year, d_moy@4 as d_moy, avg_monthly_sales@6 as avg_monthly_sales, sum_sales@5 as sum_sales, sum_sales@7 as psum, sum_sales@8 as nsum] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_category@0, i_category@0), (i_brand@1, i_brand@1), (cc_name@2, cc_name@2), (CAST(v1.rn AS Decimal128(21, 0))@9, v1_lead.rn - Decimal128(Some(1),20,0)@5)], projection=[i_category@0, i_brand@1, cc_name@2, d_year@3, d_moy@4, sum_sales@5, avg_monthly_sales@6, sum_sales@8, sum_sales@13] - │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec │ ProjectionExec: expr=[i_category@0 as i_category, i_brand@1 as i_brand, cc_name@2 as cc_name, d_year@3 as d_year, d_moy@4 as d_moy, sum_sales@5 as sum_sales, avg_monthly_sales@6 as avg_monthly_sales, rn@7 as rn, sum_sales@8 as sum_sales, CAST(rn@7 AS Decimal128(21, 0)) as CAST(v1.rn AS Decimal128(21, 0))] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_category@0, i_category@0), (i_brand@1, i_brand@1), (cc_name@2, cc_name@2), (CAST(v1.rn AS Decimal128(21, 0))@8, v1_lag.rn + Decimal128(Some(1),20,0)@5)], projection=[i_category@0, i_brand@1, cc_name@2, d_year@3, d_moy@4, sum_sales@5, avg_monthly_sales@6, rn@7, sum_sales@12] - │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec │ ProjectionExec: expr=[i_category@0 as i_category, i_brand@1 as i_brand, cc_name@2 as cc_name, d_year@3 as d_year, d_moy@4 as d_moy, sum(catalog_sales.cs_sales_price)@5 as sum_sales, avg(sum(catalog_sales.cs_sales_price)) PARTITION BY [item.i_category, item.i_brand, call_center.cc_name, date_dim.d_year] ORDER BY [date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as avg_monthly_sales, rank() PARTITION BY [item.i_category, item.i_brand, call_center.cc_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 as rn, CAST(rank() PARTITION BY [item.i_category, item.i_brand, call_center.cc_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 AS Decimal128(21, 0)) as CAST(v1.rn AS Decimal128(21, 0))] │ FilterExec: d_year@3 = 1999 AND avg(sum(catalog_sales.cs_sales_price)) PARTITION BY [item.i_category, item.i_brand, call_center.cc_name, date_dim.d_year] ORDER BY [date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 > Some(0),19,6 AND CASE WHEN avg(sum(catalog_sales.cs_sales_price)) PARTITION BY [item.i_category, item.i_brand, call_center.cc_name, date_dim.d_year] ORDER BY [date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 > Some(0),19,6 THEN abs(sum(catalog_sales.cs_sales_price)@5 - avg(sum(catalog_sales.cs_sales_price)) PARTITION BY [item.i_category, item.i_brand, call_center.cc_name, date_dim.d_year] ORDER BY [date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6) / avg(sum(catalog_sales.cs_sales_price)) PARTITION BY [item.i_category, item.i_brand, call_center.cc_name, date_dim.d_year] ORDER BY [date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 END > Some(1000000000),30,10 │ BoundedWindowAggExec: wdw=[rank() PARTITION BY [item.i_category, item.i_brand, call_center.cc_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [item.i_category, item.i_brand, call_center.cc_name] ORDER BY [date_dim.d_year ASC NULLS LAST, date_dim.d_moy ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] @@ -5884,35 +5901,35 @@ mod tests { │ AggregateExec: mode=Partial, gby=[i_category@1 as i_category, i_brand@0 as i_brand, cc_name@5 as cc_name, d_year@3 as d_year, d_moy@4 as d_moy], aggr=[sum(catalog_sales.cs_sales_price)] │ ProjectionExec: expr=[i_brand@1 as i_brand, i_category@2 as i_category, cs_sales_price@3 as cs_sales_price, d_year@4 as d_year, d_moy@5 as d_moy, cc_name@0 as cc_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(call_center.cc_call_center_sk AS Float64)@2, cs_call_center_sk@2)], projection=[cc_name@1, i_brand@3, i_category@4, cs_sales_price@6, d_year@7, d_moy@8] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[i_brand@2 as i_brand, i_category@3 as i_category, cs_call_center_sk@4 as cs_call_center_sk, cs_sales_price@5 as cs_sales_price, d_year@0 as d_year, d_moy@1 as d_moy] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, cs_sold_date_sk@2)], projection=[d_year@1, d_moy@2, i_brand@4, i_category@5, cs_call_center_sk@7, cs_sales_price@8] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@2)], projection=[i_brand@1, i_category@2, cs_sold_date_sk@3, cs_call_center_sk@4, cs_sales_price@6] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_call_center_sk, cs_item_sk, cs_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/call_center/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-3.parquet]]}, projection=[cc_call_center_sk, cc_name, CAST(cc_call_center_sk@0 AS Float64) as CAST(call_center.cc_call_center_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 1999 OR d_year@1 = 1998 AND d_moy@2 = 12 OR d_year@1 = 2000 AND d_moy@2 = 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_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1999 OR d_year@6 = 1998 AND d_moy@8 = 12 OR d_year@6 = 2000 AND d_moy@8 = 1, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR 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 <= 12 AND 12 <= d_moy_max@5 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 1 AND 1 <= d_moy_max@5, required_guarantees=[d_year in (1998, 1999, 2000)] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/call_center/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-3.parquet]]}, projection=[cc_call_center_sk, cc_name, CAST(cc_call_center_sk@0 AS Float64) as CAST(call_center.cc_call_center_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 1999 OR d_year@1 = 1998 AND d_moy@2 = 12 OR d_year@1 = 2000 AND d_moy@2 = 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_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1999 OR d_year@6 = 1998 AND d_moy@8 = 12 OR d_year@6 = 2000 AND d_moy@8 = 1, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR 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 <= 12 AND 12 <= d_moy_max@5 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 1 AND 1 <= d_moy_max@5, required_guarantees=[d_year in (1998, 1999, 2000)] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, cc_name@2], 3), input_partitions=3 @@ -5924,35 +5941,35 @@ mod tests { │ AggregateExec: mode=Partial, gby=[i_category@1 as i_category, i_brand@0 as i_brand, cc_name@5 as cc_name, d_year@3 as d_year, d_moy@4 as d_moy], aggr=[sum(catalog_sales.cs_sales_price)] │ ProjectionExec: expr=[i_brand@1 as i_brand, i_category@2 as i_category, cs_sales_price@3 as cs_sales_price, d_year@4 as d_year, d_moy@5 as d_moy, cc_name@0 as cc_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(call_center.cc_call_center_sk AS Float64)@2, cs_call_center_sk@2)], projection=[cc_name@1, i_brand@3, i_category@4, cs_sales_price@6, d_year@7, d_moy@8] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[i_brand@2 as i_brand, i_category@3 as i_category, cs_call_center_sk@4 as cs_call_center_sk, cs_sales_price@5 as cs_sales_price, d_year@0 as d_year, d_moy@1 as d_moy] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, cs_sold_date_sk@2)], projection=[d_year@1, d_moy@2, i_brand@4, i_category@5, cs_call_center_sk@7, cs_sales_price@8] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@2)], projection=[i_brand@1, i_category@2, cs_sold_date_sk@3, cs_call_center_sk@4, cs_sales_price@6] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_call_center_sk, cs_item_sk, cs_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/call_center/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-3.parquet]]}, projection=[cc_call_center_sk, cc_name, CAST(cc_call_center_sk@0 AS Float64) as CAST(call_center.cc_call_center_sk AS Float64)], file_type=parquet + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/call_center/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-3.parquet]]}, projection=[cc_call_center_sk, cc_name, CAST(cc_call_center_sk@0 AS Float64) as CAST(call_center.cc_call_center_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 1999 OR d_year@1 = 1998 AND d_moy@2 = 12 OR d_year@1 = 2000 AND d_moy@2 = 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_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1999 OR d_year@6 = 1998 AND d_moy@8 = 12 OR d_year@6 = 2000 AND d_moy@8 = 1, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR 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 <= 12 AND 12 <= d_moy_max@5 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 1 AND 1 <= d_moy_max@5, required_guarantees=[d_year in (1998, 1999, 2000)] + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 1999 OR d_year@1 = 1998 AND d_moy@2 = 12 OR d_year@1 = 2000 AND d_moy@2 = 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_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1999 OR d_year@6 = 1998 AND d_moy@8 = 12 OR d_year@6 = 2000 AND d_moy@8 = 1, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR 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 <= 12 AND 12 <= d_moy_max@5 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 1 AND 1 <= d_moy_max@5, required_guarantees=[d_year in (1998, 1999, 2000)] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 15 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ RepartitionExec: partitioning=Hash([i_category@0, i_brand@1, cc_name@2], 3), input_partitions=3 @@ -5964,35 +5981,35 @@ mod tests { │ AggregateExec: mode=Partial, gby=[i_category@1 as i_category, i_brand@0 as i_brand, cc_name@5 as cc_name, d_year@3 as d_year, d_moy@4 as d_moy], aggr=[sum(catalog_sales.cs_sales_price)] │ ProjectionExec: expr=[i_brand@1 as i_brand, i_category@2 as i_category, cs_sales_price@3 as cs_sales_price, d_year@4 as d_year, d_moy@5 as d_moy, cc_name@0 as cc_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(call_center.cc_call_center_sk AS Float64)@2, cs_call_center_sk@2)], projection=[cc_name@1, i_brand@3, i_category@4, cs_sales_price@6, d_year@7, d_moy@8] - │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[i_brand@2 as i_brand, i_category@3 as i_category, cs_call_center_sk@4 as cs_call_center_sk, cs_sales_price@5 as cs_sales_price, d_year@0 as d_year, d_moy@1 as d_moy] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, cs_sold_date_sk@2)], projection=[d_year@1, d_moy@2, i_brand@4, i_category@5, cs_call_center_sk@7, cs_sales_price@8] - │ CoalescePartitionsExec - │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@2)], projection=[i_brand@1, i_category@2, cs_sold_date_sk@3, cs_call_center_sk@4, cs_sales_price@6] - │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_call_center_sk, cs_item_sk, cs_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/call_center/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-3.parquet]]}, projection=[cc_call_center_sk, cc_name, CAST(cc_call_center_sk@0 AS Float64) as CAST(call_center.cc_call_center_sk AS Float64)], file_type=parquet + ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/call_center/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-3.parquet]]}, projection=[cc_call_center_sk, cc_name, CAST(cc_call_center_sk@0 AS Float64) as CAST(call_center.cc_call_center_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 1999 OR d_year@1 = 1998 AND d_moy@2 = 12 OR d_year@1 = 2000 AND d_moy@2 = 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_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1999 OR d_year@6 = 1998 AND d_moy@8 = 12 OR d_year@6 = 2000 AND d_moy@8 = 1, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR 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 <= 12 AND 12 <= d_moy_max@5 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 1 AND 1 <= d_moy_max@5, required_guarantees=[d_year in (1998, 1999, 2000)] + ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 1999 OR d_year@1 = 1998 AND d_moy@2 = 12 OR d_year@1 = 2000 AND d_moy@2 = 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_date_sk, d_year, d_moy], file_type=parquet, predicate=d_year@6 = 1999 OR d_year@6 = 1998 AND d_moy@8 = 12 OR d_year@6 = 2000 AND d_moy@8 = 1, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1 OR 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 <= 12 AND 12 <= d_moy_max@5 OR d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2000 AND 2000 <= d_year_max@1 AND d_moy_null_count@6 != row_count@3 AND d_moy_min@4 <= 1 AND 1 <= d_moy_max@5, required_guarantees=[d_year in (1998, 1999, 2000)] └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet + ┌───── Stage 13 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_category], file_type=parquet └────────────────────────────────────────────────── "#); Ok(()) @@ -6007,8 +6024,8 @@ mod tests { │ ProjectionExec: expr=[item_id@1 as item_id, ss_item_rev@2 as ss_item_rev, ss_item_rev@2 / __common_expr_7@0 * Some(100),20,0 as ss_dev, cs_item_rev@3 as cs_item_rev, cs_item_rev@3 / __common_expr_7@0 * Some(100),20,0 as cs_dev, ws_item_rev@4 as ws_item_rev, ws_item_rev@4 / __common_expr_7@0 * Some(100),20,0 as ws_dev, __common_expr_7@0 as average] │ ProjectionExec: expr=[(ss_item_rev@2 + cs_item_rev@3 + ws_item_rev@0) / Some(3),20,0 as __common_expr_7, item_id@1 as item_id, ss_item_rev@2 as ss_item_rev, cs_item_rev@3 as cs_item_rev, ws_item_rev@0 as ws_item_rev] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(item_id@0, item_id@0)], filter=CAST(ss_item_rev@0 AS Decimal128(30, 15)) >= CAST(0.9 * CAST(ws_item_rev@2 AS Float64) AS Decimal128(30, 15)) AND CAST(ss_item_rev@0 AS Decimal128(30, 15)) <= CAST(1.1 * CAST(ws_item_rev@2 AS Float64) AS Decimal128(30, 15)) AND CAST(cs_item_rev@1 AS Decimal128(30, 15)) >= CAST(0.9 * CAST(ws_item_rev@2 AS Float64) AS Decimal128(30, 15)) AND CAST(cs_item_rev@1 AS Decimal128(30, 15)) <= CAST(1.1 * CAST(ws_item_rev@2 AS Float64) AS Decimal128(30, 15)) AND CAST(ws_item_rev@2 AS Decimal128(30, 15)) >= CAST(0.9 * CAST(ss_item_rev@0 AS Float64) AS Decimal128(30, 15)) AND CAST(ws_item_rev@2 AS Decimal128(30, 15)) <= CAST(1.1 * CAST(ss_item_rev@0 AS Float64) AS Decimal128(30, 15)) AND CAST(ws_item_rev@2 AS Decimal128(30, 15)) >= CAST(0.9 * CAST(cs_item_rev@1 AS Float64) AS Decimal128(30, 15)) AND CAST(ws_item_rev@2 AS Decimal128(30, 15)) <= CAST(1.1 * CAST(cs_item_rev@1 AS Float64) AS Decimal128(30, 15)), projection=[ws_item_rev@1, item_id@2, ss_item_rev@3, cs_item_rev@4] - │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec │ ProjectionExec: expr=[i_item_id@0 as item_id, sum(web_sales.ws_ext_sales_price)@1 as ws_item_rev] │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(web_sales.ws_ext_sales_price)] │ RepartitionExec: partitioning=Hash([i_item_id@0], 3), input_partitions=3 @@ -6017,8 +6034,8 @@ mod tests { │ CoalescePartitionsExec │ [Stage 4] => NetworkCoalesceExec: output_partitions=9, 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 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec │ FilterExec: d_date@0 = 2000-01-03, projection=[d_week_seq@1] │ 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=d_date@2 = 2000-01-03, pruning_predicate=d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-01-03 AND 2000-01-03 <= d_date_max@1, required_guarantees=[d_date in (2000-01-03)] @@ -6026,8 +6043,8 @@ mod tests { │ DataSourceExec: file_groups={2 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet, /testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date, d_week_seq], file_type=parquet, predicate=DynamicFilter [ empty ] │ ProjectionExec: expr=[item_id@1 as item_id, ss_item_rev@2 as ss_item_rev, cs_item_rev@0 as cs_item_rev] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(item_id@0, item_id@0)], filter=CAST(ss_item_rev@0 AS Decimal128(30, 15)) >= CAST(0.9 * CAST(cs_item_rev@1 AS Float64) AS Decimal128(30, 15)) AND CAST(ss_item_rev@0 AS Decimal128(30, 15)) <= CAST(1.1 * CAST(cs_item_rev@1 AS Float64) AS Decimal128(30, 15)) AND CAST(cs_item_rev@1 AS Decimal128(30, 15)) >= CAST(0.9 * CAST(ss_item_rev@0 AS Float64) AS Decimal128(30, 15)) AND CAST(cs_item_rev@1 AS Decimal128(30, 15)) <= CAST(1.1 * CAST(ss_item_rev@0 AS Float64) AS Decimal128(30, 15)), projection=[cs_item_rev@1, item_id@2, ss_item_rev@3] - │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec │ ProjectionExec: expr=[i_item_id@0 as item_id, sum(catalog_sales.cs_ext_sales_price)@1 as cs_item_rev] │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(catalog_sales.cs_ext_sales_price)] │ RepartitionExec: partitioning=Hash([i_item_id@0], 3), input_partitions=3 @@ -6036,8 +6053,8 @@ mod tests { │ CoalescePartitionsExec │ [Stage 8] => NetworkCoalesceExec: output_partitions=9, 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 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec │ FilterExec: d_date@0 = 2000-01-03, projection=[d_week_seq@1] │ 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=d_date@2 = 2000-01-03, pruning_predicate=d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-01-03 AND 2000-01-03 <= d_date_max@1, required_guarantees=[d_date in (2000-01-03)] @@ -6051,8 +6068,8 @@ mod tests { │ CoalescePartitionsExec │ [Stage 12] => NetworkCoalesceExec: output_partitions=9, 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 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec │ FilterExec: d_date@0 = 2000-01-03, projection=[d_week_seq@1] │ 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=d_date@2 = 2000-01-03, pruning_predicate=d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-01-03 AND 2000-01-03 <= d_date_max@1, required_guarantees=[d_date in (2000-01-03)] @@ -6076,15 +6093,15 @@ mod tests { │ RepartitionExec: partitioning=Hash([ws_sold_date_sk@0], 9), input_partitions=2 │ ProjectionExec: expr=[ws_sold_date_sk@1 as ws_sold_date_sk, ws_ext_sales_price@2 as ws_ext_sales_price, i_item_id@0 as i_item_id] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_item_id@1, ws_sold_date_sk@2, ws_ext_sales_price@4] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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] @@ -6103,15 +6120,15 @@ mod tests { │ RepartitionExec: partitioning=Hash([cs_sold_date_sk@0], 9), input_partitions=2 │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_ext_sales_price@2 as cs_ext_sales_price, i_item_id@0 as i_item_id] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_item_id@1, cs_sold_date_sk@2, cs_ext_sales_price@4] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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] @@ -6130,15 +6147,15 @@ mod tests { │ RepartitionExec: partitioning=Hash([ss_sold_date_sk@0], 9), input_partitions=2 │ ProjectionExec: expr=[ss_sold_date_sk@1 as ss_sold_date_sk, ss_ext_sales_price@2 as ss_ext_sales_price, i_item_id@0 as i_item_id] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_item_id@1, ss_sold_date_sk@2, ss_ext_sales_price@4] - │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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 10 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 └────────────────────────────────────────────────── "); Ok(()) @@ -6155,91 +6172,91 @@ mod tests { │ 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 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 - │ ProjectionExec: expr=[s_store_name@9 as s_store_name1, d_week_seq@0 as d_week_seq1, s_store_id@8 as s_store_id1, sun_sales@1 as sun_sales1, mon_sales@2 as mon_sales1, tue_sales@3 as tue_sales1, wed_sales@4 as wed_sales1, thu_sales@5 as thu_sales1, fri_sales@6 as fri_sales1, sat_sales@7 as sat_sales1, CAST(d_week_seq@0 AS Int64) as CAST(y.d_week_seq1 AS Int64)] - │ 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, s_store_name@9] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ FilterExec: d_month_seq@0 >= 1212 AND d_month_seq@0 <= 1223, 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 >= 1212 AND d_month_seq@3 <= 1223 AND DynamicFilter [ empty ], pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1212 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1223, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ ProjectionExec: expr=[d_week_seq@2 as d_week_seq, sun_sales@3 as sun_sales, mon_sales@4 as mon_sales, tue_sales@5 as tue_sales, wed_sales@6 as wed_sales, thu_sales@7 as thu_sales, fri_sales@8 as fri_sales, sat_sales@9 as sat_sales, s_store_id@0 as s_store_id, s_store_name@1 as s_store_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@1)], projection=[s_store_id@1, s_store_name@2, d_week_seq@4, sun_sales@6, mon_sales@7, tue_sales@8, wed_sales@9, thu_sales@10, fri_sales@11, sat_sales@12] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=2 - │ ProjectionExec: expr=[d_week_seq@0 as d_week_seq, ss_store_sk@1 as ss_store_sk, sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END)@2 as sun_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END)@3 as mon_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END)@4 as tue_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END)@5 as wed_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END)@6 as thu_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END)@7 as fri_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)@8 as sat_sales] - │ AggregateExec: mode=FinalPartitioned, gby=[d_week_seq@0 as d_week_seq, ss_store_sk@1 as ss_store_sk], aggr=[sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] - │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, s_store_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + ┌───── Stage 5 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_name@9 as s_store_name1, d_week_seq@0 as d_week_seq1, s_store_id@8 as s_store_id1, sun_sales@1 as sun_sales1, mon_sales@2 as mon_sales1, tue_sales@3 as tue_sales1, wed_sales@4 as wed_sales1, thu_sales@5 as thu_sales1, fri_sales@6 as fri_sales1, sat_sales@7 as sat_sales1, CAST(d_week_seq@0 AS Int64) as CAST(y.d_week_seq1 AS Int64)] + │ 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, s_store_name@9] + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, input_tasks=2 + │ FilterExec: d_month_seq@0 >= 1212 AND d_month_seq@0 <= 1223, 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 >= 1212 AND d_month_seq@3 <= 1223 AND DynamicFilter [ empty ], pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1212 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1223, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_week_seq@2 as d_week_seq, sun_sales@3 as sun_sales, mon_sales@4 as mon_sales, tue_sales@5 as tue_sales, wed_sales@6 as wed_sales, thu_sales@7 as thu_sales, fri_sales@8 as fri_sales, sat_sales@9 as sat_sales, s_store_id@0 as s_store_id, s_store_name@1 as s_store_name] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@1)], projection=[s_store_id@1, s_store_name@2, d_week_seq@4, sun_sales@6, mon_sales@7, tue_sales@8, wed_sales@9, thu_sales@10, fri_sales@11, sat_sales@12] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, input_tasks=2 + │ ProjectionExec: expr=[d_week_seq@0 as d_week_seq, ss_store_sk@1 as ss_store_sk, sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END)@2 as sun_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END)@3 as mon_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END)@4 as tue_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END)@5 as wed_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END)@6 as thu_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END)@7 as fri_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)@8 as sat_sales] + │ AggregateExec: mode=FinalPartitioned, gby=[d_week_seq@0 as d_week_seq, ss_store_sk@1 as ss_store_sk], aggr=[sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)] + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, s_store_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([d_week_seq@0, ss_store_sk@1], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[d_week_seq@2 as d_week_seq, ss_store_sk@0 as ss_store_sk], aggr=[sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)] │ ProjectionExec: expr=[ss_store_sk@2 as ss_store_sk, ss_sales_price@3 as ss_sales_price, d_week_seq@0 as d_week_seq, d_day_name@1 as d_day_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ss_sold_date_sk@0)], projection=[d_week_seq@1, d_day_name@2, ss_store_sk@5, ss_sales_price@6] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_week_seq@1 as d_week_seq, d_day_name@2 as d_day_name, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_week_seq, d_day_name], file_type=parquet + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_week_seq@1 as d_week_seq, d_day_name@2 as d_day_name, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_week_seq, d_day_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ ProjectionExec: expr=[d_week_seq@1 as d_week_seq, sun_sales@2 as sun_sales, mon_sales@3 as mon_sales, tue_sales@4 as tue_sales, wed_sales@5 as wed_sales, thu_sales@6 as thu_sales, fri_sales@7 as fri_sales, sat_sales@8 as sat_sales, s_store_id@0 as s_store_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, ss_store_sk@1)], projection=[s_store_id@1, d_week_seq@3, sun_sales@5, mon_sales@6, tue_sales@7, wed_sales@8, thu_sales@9, fri_sales@10, sat_sales@11] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=2 - │ ProjectionExec: expr=[d_week_seq@0 as d_week_seq, ss_store_sk@1 as ss_store_sk, sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END)@2 as sun_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END)@3 as mon_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END)@4 as tue_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END)@5 as wed_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END)@6 as thu_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END)@7 as fri_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)@8 as sat_sales] - │ AggregateExec: mode=FinalPartitioned, gby=[d_week_seq@0 as d_week_seq, ss_store_sk@1 as ss_store_sk], aggr=[sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)] - │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p3] t1:[p4..p7] - │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + ┌───── Stage 9 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_week_seq@1 as d_week_seq, sun_sales@2 as sun_sales, mon_sales@3 as mon_sales, tue_sales@4 as tue_sales, wed_sales@5 as wed_sales, thu_sales@6 as thu_sales, fri_sales@7 as fri_sales, sat_sales@8 as sat_sales, s_store_id@0 as s_store_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, ss_store_sk@1)], projection=[s_store_id@1, d_week_seq@3, sun_sales@5, mon_sales@6, tue_sales@7, wed_sales@8, thu_sales@9, fri_sales@10, sat_sales@11] + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, input_tasks=2 + │ ProjectionExec: expr=[d_week_seq@0 as d_week_seq, ss_store_sk@1 as ss_store_sk, sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END)@2 as sun_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END)@3 as mon_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END)@4 as tue_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END)@5 as wed_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END)@6 as thu_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END)@7 as fri_sales, sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)@8 as sat_sales] + │ AggregateExec: mode=FinalPartitioned, gby=[d_week_seq@0 as d_week_seq, ss_store_sk@1 as ss_store_sk], aggr=[sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)] + │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([d_week_seq@0, ss_store_sk@1], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[d_week_seq@2 as d_week_seq, ss_store_sk@0 as ss_store_sk], aggr=[sum(CASE WHEN date_dim.d_day_name = Utf8("Sunday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Monday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Tuesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Wednesday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Thursday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Friday") THEN store_sales.ss_sales_price ELSE NULL END), sum(CASE WHEN date_dim.d_day_name = Utf8("Saturday") THEN store_sales.ss_sales_price ELSE NULL END)] │ ProjectionExec: expr=[ss_store_sk@2 as ss_store_sk, ss_sales_price@3 as ss_sales_price, d_week_seq@0 as d_week_seq, d_day_name@1 as d_day_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@3, ss_sold_date_sk@0)], projection=[d_week_seq@1, d_day_name@2, ss_store_sk@5, ss_sales_price@6] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_week_seq@1 as d_week_seq, d_day_name@2 as d_day_name, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_week_seq, d_day_name], file_type=parquet + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_week_seq@1 as d_week_seq, d_day_name@2 as d_day_name, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_week_seq, d_day_name], file_type=parquet └────────────────────────────────────────────────── "#); Ok(()) @@ -6287,108 +6304,108 @@ mod tests { └────────────────────────────────────────────────── ┌───── 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 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[ss_item_sk@0, ss_ext_sales_price@2] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_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/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_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[ss_item_sk@0, ss_ext_sales_price@2] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] + │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_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/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_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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_addr_sk@4, ss_ext_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 1] => 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_addr_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ 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_addr_sk@4, ss_ext_sales_price@5] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_addr_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 = 1998 AND d_moy@2 = 9, projection=[d_date_sk@0] - │ 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_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 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 = 1998 AND d_moy@2 = 9, projection=[d_date_sk@0] + │ 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_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 + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_bill_addr_sk@0, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[cs_item_sk@1, cs_ext_sales_price@2] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_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/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_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_bill_addr_sk@0, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[cs_item_sk@1, cs_ext_sales_price@2] + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] + │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_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/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_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_bill_addr_sk@3, cs_item_sk@4, cs_ext_sales_price@5] - │ 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_bill_addr_sk, cs_item_sk, cs_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_bill_addr_sk@3, cs_item_sk@4, cs_ext_sales_price@5] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_addr_sk, cs_item_sk, cs_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 = 1998 AND d_moy@2 = 9, projection=[d_date_sk@0] - │ 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_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 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 = 1998 AND d_moy@2 = 9, projection=[d_date_sk@0] + │ 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_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 + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_bill_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[ws_item_sk@0, ws_ext_sales_price@2] - │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_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/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_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_bill_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@1)], projection=[ws_item_sk@0, ws_ext_sales_price@2] + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] + │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_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/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_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2, pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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_bill_addr_sk@4, ws_ext_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 9] => 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_bill_addr_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ 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_bill_addr_sk@4, ws_ext_sales_price@5] + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_addr_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 = 1998 AND d_moy@2 = 9, projection=[d_date_sk@0] - │ 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_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 9 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 = 1998 AND d_moy@2 = 9, projection=[d_date_sk@0] + │ 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_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)] └────────────────────────────────────────────────── "); Ok(()) @@ -6413,127 +6430,127 @@ mod tests { ┌───── 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 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@2, ca_address_sk@0)], projection=[ss_item_sk@0, ss_ext_sales_price@1] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_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/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_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2 AND DynamicFilter [ empty ], pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_ext_sales_price@2, c_current_addr_sk@4] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk], file_type=parquet + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@2, ca_address_sk@0)], projection=[ss_item_sk@0, ss_ext_sales_price@1] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_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/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_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2 AND DynamicFilter [ empty ], pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_ext_sales_price@2, c_current_addr_sk@4] + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@1)], projection=[ss_item_sk@1, ss_customer_sk@2, ss_ext_sales_price@3] - │ CoalescePartitionsExec - │ [Stage 3] => 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 = 1998 AND d_moy@2 = 11, 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, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 11, 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 <= 11 AND 11 <= d_moy_max@5, required_guarantees=[d_moy in (11), d_year in (1998)] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_promo_sk@3, CAST(promotion.p_promo_sk AS Float64)@1)], projection=[ss_sold_date_sk@0, ss_item_sk@1, ss_customer_sk@2, ss_ext_sales_price@4] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[p_promo_sk@0 as p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)] - │ FilterExec: p_channel_dmail@1 = Y OR p_channel_email@2 = Y OR p_channel_tv@3 = Y, projection=[p_promo_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/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, p_channel_dmail, p_channel_email, p_channel_tv], file_type=parquet, predicate=p_channel_dmail@8 = Y OR p_channel_email@9 = Y OR p_channel_tv@11 = Y, pruning_predicate=p_channel_dmail_null_count@2 != row_count@3 AND p_channel_dmail_min@0 <= Y AND Y <= p_channel_dmail_max@1 OR p_channel_email_null_count@6 != row_count@3 AND p_channel_email_min@4 <= Y AND Y <= p_channel_email_max@5 OR p_channel_tv_null_count@9 != row_count@3 AND p_channel_tv_min@7 <= Y AND Y <= p_channel_tv_max@8, required_guarantees=[] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@1)], projection=[ss_item_sk@1, ss_customer_sk@2, ss_ext_sales_price@3] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 = 1998 AND d_moy@2 = 11, 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, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 11, 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 <= 11 AND 11 <= d_moy_max@5, required_guarantees=[d_moy in (11), d_year in (1998)] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_promo_sk@3, CAST(promotion.p_promo_sk AS Float64)@1)], projection=[ss_sold_date_sk@0, ss_item_sk@1, ss_customer_sk@2, ss_ext_sales_price@4] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[p_promo_sk@0 as p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)] + │ FilterExec: p_channel_dmail@1 = Y OR p_channel_email@2 = Y OR p_channel_tv@3 = Y, projection=[p_promo_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/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, p_channel_dmail, p_channel_email, p_channel_tv], file_type=parquet, predicate=p_channel_dmail@8 = Y OR p_channel_email@9 = Y OR p_channel_tv@11 = Y, pruning_predicate=p_channel_dmail_null_count@2 != row_count@3 AND p_channel_dmail_min@0 <= Y AND Y <= p_channel_dmail_max@1 OR p_channel_email_null_count@6 != row_count@3 AND p_channel_email_min@4 <= Y AND Y <= p_channel_email_max@5 OR p_channel_tv_null_count@9 != row_count@3 AND p_channel_tv_min@7 <= Y AND Y <= p_channel_tv_max@8, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@3)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_customer_sk@4, ss_promo_sk@6, ss_ext_sales_price@7] - │ CoalescePartitionsExec - │ [Stage 1] => 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_customer_sk, ss_store_sk, ss_promo_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@3)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_customer_sk@4, ss_promo_sk@6, ss_ext_sales_price@7] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_customer_sk, ss_store_sk, ss_promo_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_gmt_offset@1 = Some(-500),3,2, projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_gmt_offset@1 = Some(-500),3,2, projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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 + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@2, ca_address_sk@0)], projection=[ss_item_sk@0, ss_ext_sales_price@1] - │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_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/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_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2 AND DynamicFilter [ empty ], pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] - └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_ext_sales_price@2, c_current_addr_sk@4] - │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk], file_type=parquet + ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@2, ca_address_sk@0)], projection=[ss_item_sk@0, ss_ext_sales_price@1] + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ FilterExec: ca_gmt_offset@1 = Some(-500),4,2, projection=[ca_address_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/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_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-500),4,2 AND DynamicFilter [ empty ], pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-500),4,2 AND Some(-500),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-500),4,2)] + └────────────────────────────────────────────────── + ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_ext_sales_price@2, c_current_addr_sk@4] + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@1)], projection=[ss_item_sk@1, ss_customer_sk@2, ss_ext_sales_price@3] - │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, 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 = 1998 AND d_moy@2 = 11, 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, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 11, 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 <= 11 AND 11 <= d_moy_max@5, required_guarantees=[d_moy in (11), d_year in (1998)] - └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@3)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_customer_sk@4, ss_ext_sales_price@6] - │ 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/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_customer_sk, ss_store_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@1)], projection=[ss_item_sk@1, ss_customer_sk@2, ss_ext_sales_price@3] + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 = 1998 AND d_moy@2 = 11, 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, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 11, 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 <= 11 AND 11 <= d_moy_max@5, required_guarantees=[d_moy in (11), d_year in (1998)] + └────────────────────────────────────────────────── + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@3)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_customer_sk@4, ss_ext_sales_price@6] + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_customer_sk, ss_store_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_gmt_offset@1 = Some(-500),3,2, projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_gmt_offset@1 = Some(-500),3,2, projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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)] └────────────────────────────────────────────────── "); Ok(()) @@ -6557,43 +6574,43 @@ mod tests { │ AggregateExec: mode=Partial, gby=[w_substr@1 as w_substr, sm_type@2 as sm_type, web_name@3 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)] │ ProjectionExec: expr=[ws_ship_date_sk@1 - ws_sold_date_sk@0 as __common_expr_1, w_substr@2 as w_substr, sm_type@3 as sm_type, web_name@4 as web_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_ship_date_sk@1, CAST(date_dim.d_date_sk AS Float64)@1)], projection=[ws_sold_date_sk@0, ws_ship_date_sk@1, w_substr@2, sm_type@3, web_name@4] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_month_seq@1 >= 1200 AND d_month_seq@1 <= 1211, 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_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_web_site_sk@2, CAST(web_site.web_site_sk AS Float64)@2)], projection=[ws_sold_date_sk@0, ws_ship_date_sk@1, w_substr@3, sm_type@4, web_name@6] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_site/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-3.parquet]]}, projection=[web_site_sk, web_name, CAST(web_site_sk@0 AS Float64) as CAST(web_site.web_site_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_ship_mode_sk@3, CAST(ship_mode.sm_ship_mode_sk AS Float64)@2)], projection=[ws_sold_date_sk@0, ws_ship_date_sk@1, ws_web_site_sk@2, w_substr@4, sm_type@6] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-3.parquet]]}, projection=[sm_ship_mode_sk, sm_type, CAST(sm_ship_mode_sk@0 AS Float64) as CAST(ship_mode.sm_ship_mode_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ ProjectionExec: expr=[ws_sold_date_sk@1 as ws_sold_date_sk, ws_ship_date_sk@2 as ws_ship_date_sk, ws_web_site_sk@3 as ws_web_site_sk, ws_ship_mode_sk@4 as ws_ship_mode_sk, w_substr@0 as w_substr] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(sq1.w_warehouse_sk AS Float64)@2, ws_warehouse_sk@4)], projection=[w_substr@0, ws_sold_date_sk@3, ws_ship_date_sk@4, ws_web_site_sk@5, ws_ship_mode_sk@6] - │ CoalescePartitionsExec - │ [Stage 1] => 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_ship_date_sk, ws_web_site_sk, ws_ship_mode_sk, ws_warehouse_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_web_site_sk@2, CAST(web_site.web_site_sk AS Float64)@2)], projection=[ws_sold_date_sk@0, ws_ship_date_sk@1, w_substr@3, sm_type@4, web_name@6] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_site/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-3.parquet]]}, projection=[web_site_sk, web_name, CAST(web_site_sk@0 AS Float64) as CAST(web_site.web_site_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_ship_mode_sk@3, CAST(ship_mode.sm_ship_mode_sk AS Float64)@2)], projection=[ws_sold_date_sk@0, ws_ship_date_sk@1, ws_web_site_sk@2, w_substr@4, sm_type@6] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-3.parquet]]}, projection=[sm_ship_mode_sk, sm_type, CAST(sm_ship_mode_sk@0 AS Float64) as CAST(ship_mode.sm_ship_mode_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ws_sold_date_sk@1 as ws_sold_date_sk, ws_ship_date_sk@2 as ws_ship_date_sk, ws_web_site_sk@3 as ws_web_site_sk, ws_ship_mode_sk@4 as ws_ship_mode_sk, w_substr@0 as w_substr] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(sq1.w_warehouse_sk AS Float64)@2, ws_warehouse_sk@4)], projection=[w_substr@0, ws_sold_date_sk@3, ws_ship_date_sk@4, ws_web_site_sk@5, ws_ship_mode_sk@6] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_ship_date_sk, ws_web_site_sk, ws_ship_mode_sk, ws_warehouse_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[substr(w_warehouse_name@2, 1, 20) as w_substr, w_warehouse_sk, CAST(w_warehouse_sk@0 AS Float64) as CAST(sq1.w_warehouse_sk AS Float64)], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[substr(w_warehouse_name@2, 1, 20) as w_substr, w_warehouse_sk, CAST(w_warehouse_sk@0 AS Float64) as CAST(sq1.w_warehouse_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── "); Ok(()) @@ -6621,37 +6638,37 @@ mod tests { │ RepartitionExec: partitioning=Hash([i_manager_id@0, d_moy@1], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[i_manager_id@0 as i_manager_id, d_moy@2 as d_moy], aggr=[sum(store_sales.ss_sales_price)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@1)], projection=[i_manager_id@2, ss_sales_price@4, d_moy@5] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[i_manager_id@1 as i_manager_id, ss_store_sk@2 as ss_store_sk, ss_sales_price@3 as ss_sales_price, d_moy@0 as d_moy] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@1)], projection=[d_moy@1, i_manager_id@3, ss_store_sk@5, ss_sales_price@6] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_manager_id@1, ss_sold_date_sk@2, ss_store_sk@4, ss_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_moy@1 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_month_seq@1 IN (SET) ([1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211]), projection=[d_date_sk@0, d_moy@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/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, d_moy], file_type=parquet, predicate=d_month_seq@3 IN (SET) ([1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211]), pruning_predicate=d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1200 AND 1200 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1201 AND 1201 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1202 AND 1202 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1203 AND 1203 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1204 AND 1204 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1205 AND 1205 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1206 AND 1206 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1207 AND 1207 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1208 AND 1208 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1209 AND 1209 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1210 AND 1210 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1211 AND 1211 <= d_month_seq_max@1, required_guarantees=[d_month_seq in (1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_moy@1 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_month_seq@1 IN (SET) ([1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211]), projection=[d_date_sk@0, d_moy@2] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: (i_category@3 = Books OR i_category@3 = Children OR i_category@3 = Electronics) AND i_class@2 IN (SET) ([personal, portable, reference, self-help]) AND i_brand@1 IN (SET) ([scholaramalgamalg #14, scholaramalgamalg #7, exportiunivamalg #9, scholaramalgamalg #9]) OR (i_category@3 = Women OR i_category@3 = Music OR i_category@3 = Men) AND i_class@2 IN (SET) ([accessories, classical, fragrances, pants]) AND i_brand@1 IN (SET) ([amalgimporto #1, edu packscholar #1, exportiimporto #1, importoamalg #1]), projection=[i_item_sk@0, i_manager_id@4] │ 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_month_seq, d_moy], file_type=parquet, predicate=d_month_seq@3 IN (SET) ([1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211]), pruning_predicate=d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1200 AND 1200 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1201 AND 1201 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1202 AND 1202 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1203 AND 1203 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1204 AND 1204 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1205 AND 1205 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1206 AND 1206 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1207 AND 1207 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1208 AND 1208 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1209 AND 1209 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1210 AND 1210 <= d_month_seq_max@1 OR d_month_seq_null_count@2 != row_count@3 AND d_month_seq_min@0 <= 1211 AND 1211 <= d_month_seq_max@1, required_guarantees=[d_month_seq in (1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211)] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: (i_category@3 = Books OR i_category@3 = Children OR i_category@3 = Electronics) AND i_class@2 IN (SET) ([personal, portable, reference, self-help]) AND i_brand@1 IN (SET) ([scholaramalgamalg #14, scholaramalgamalg #7, exportiunivamalg #9, scholaramalgamalg #9]) OR (i_category@3 = Women OR i_category@3 = Music OR i_category@3 = Men) AND i_class@2 IN (SET) ([accessories, classical, fragrances, pants]) AND i_brand@1 IN (SET) ([amalgimporto #1, edu packscholar #1, exportiimporto #1, importoamalg #1]), projection=[i_item_sk@0, i_manager_id@4] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_class, i_category, i_manager_id], file_type=parquet, predicate=(i_category@12 = Books OR i_category@12 = Children OR i_category@12 = Electronics) AND i_class@10 IN (SET) ([personal, portable, reference, self-help]) AND i_brand@8 IN (SET) ([scholaramalgamalg #14, scholaramalgamalg #7, exportiunivamalg #9, scholaramalgamalg #9]) OR (i_category@12 = Women OR i_category@12 = Music OR i_category@12 = Men) AND i_class@10 IN (SET) ([accessories, classical, fragrances, pants]) AND i_brand@8 IN (SET) ([amalgimporto #1, edu packscholar #1, exportiimporto #1, importoamalg #1]), pruning_predicate=(i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Children AND Children <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Electronics AND Electronics <= i_category_max@1) AND (i_class_null_count@6 != row_count@3 AND i_class_min@4 <= personal AND personal <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= portable AND portable <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= reference AND reference <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= self-help AND self-help <= i_class_max@5) AND (i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= scholaramalgamalg #14 AND scholaramalgamalg #14 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= scholaramalgamalg #7 AND scholaramalgamalg #7 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= exportiunivamalg #9 AND exportiunivamalg #9 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= scholaramalgamalg #9 AND scholaramalgamalg #9 <= i_brand_max@8) OR (i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Women AND Women <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Music AND Music <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Men AND Men <= i_category_max@1) AND (i_class_null_count@6 != row_count@3 AND i_class_min@4 <= accessories AND accessories <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= classical AND classical <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= fragrances AND fragrances <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= pants AND pants <= i_class_max@5) AND (i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= amalgimporto #1 AND amalgimporto #1 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= edu packscholar #1 AND edu packscholar #1 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= exportiimporto #1 AND exportiimporto #1 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= importoamalg #1 AND importoamalg #1 <= i_brand_max@8), required_guarantees=[i_brand in (amalgimporto #1, edu packscholar #1, exportiimporto #1, exportiunivamalg #9, importoamalg #1, scholaramalgamalg #14, scholaramalgamalg #7, scholaramalgamalg #9), i_class in (accessories, classical, fragrances, pants, personal, portable, reference, self-help)] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_class, i_category, i_manager_id], file_type=parquet, predicate=(i_category@12 = Books OR i_category@12 = Children OR i_category@12 = Electronics) AND i_class@10 IN (SET) ([personal, portable, reference, self-help]) AND i_brand@8 IN (SET) ([scholaramalgamalg #14, scholaramalgamalg #7, exportiunivamalg #9, scholaramalgamalg #9]) OR (i_category@12 = Women OR i_category@12 = Music OR i_category@12 = Men) AND i_class@10 IN (SET) ([accessories, classical, fragrances, pants]) AND i_brand@8 IN (SET) ([amalgimporto #1, edu packscholar #1, exportiimporto #1, importoamalg #1]), pruning_predicate=(i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Children AND Children <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Electronics AND Electronics <= i_category_max@1) AND (i_class_null_count@6 != row_count@3 AND i_class_min@4 <= personal AND personal <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= portable AND portable <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= reference AND reference <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= self-help AND self-help <= i_class_max@5) AND (i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= scholaramalgamalg #14 AND scholaramalgamalg #14 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= scholaramalgamalg #7 AND scholaramalgamalg #7 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= exportiunivamalg #9 AND exportiunivamalg #9 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= scholaramalgamalg #9 AND scholaramalgamalg #9 <= i_brand_max@8) OR (i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Women AND Women <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Music AND Music <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Men AND Men <= i_category_max@1) AND (i_class_null_count@6 != row_count@3 AND i_class_min@4 <= accessories AND accessories <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= classical AND classical <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= fragrances AND fragrances <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= pants AND pants <= i_class_max@5) AND (i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= amalgimporto #1 AND amalgimporto #1 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= edu packscholar #1 AND edu packscholar #1 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= exportiimporto #1 AND exportiimporto #1 <= i_brand_max@8 OR i_brand_null_count@9 != row_count@3 AND i_brand_min@7 <= importoamalg #1 AND importoamalg #1 <= i_brand_max@8), required_guarantees=[i_brand in (amalgimporto #1, edu packscholar #1, exportiimporto #1, exportiunivamalg #9, importoamalg #1, scholaramalgamalg #14, scholaramalgamalg #7, scholaramalgamalg #9), i_class in (accessories, classical, fragrances, pants, personal, portable, reference, self-help)] └────────────────────────────────────────────────── "#); Ok(()) @@ -6669,170 +6686,170 @@ mod tests { │ 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 + │ [Stage 23] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 - │ ProjectionExec: expr=[i_product_name@0 as product_name, i_item_sk@1 as item_sk, s_store_name@2 as store_name, s_zip@3 as store_zip, ca_street_number@4 as b_street_number, ca_street_name@5 as b_street_name, ca_city@6 as b_city, ca_zip@7 as b_zip, ca_street_number@8 as c_street_number, ca_street_name@9 as c_street_name, ca_city@10 as c_city, ca_zip@11 as c_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 22] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 23 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[i_product_name@0 as product_name, i_item_sk@1 as item_sk, s_store_name@2 as store_name, s_zip@3 as store_zip, ca_street_number@4 as b_street_number, ca_street_name@5 as b_street_name, ca_city@6 as b_city, ca_zip@7 as b_zip, ca_street_number@8 as c_street_number, ca_street_name@9 as c_street_name, ca_city@10 as c_city, ca_zip@11 as c_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 22] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 22 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([i_product_name@0, i_item_sk@1, s_store_name@2, s_zip@3, ca_street_number@4, ca_street_name@5, ca_city@6, ca_zip@7, ca_street_number@8, ca_street_name@9, ca_city@10, ca_zip@11, d_year@12, d_year@13, d_year@14], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[i_product_name@17 as i_product_name, i_item_sk@16 as i_item_sk, s_store_name@6 as s_store_name, s_zip@7 as s_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, ca_street_number@12 as ca_street_number, ca_street_name@13 as ca_street_name, ca_city@14 as ca_city, ca_zip@15 as ca_zip, d_year@3 as d_year, d_year@4 as d_year, d_year@5 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]) │ ProjectionExec: expr=[ss_wholesale_cost@0 as ss_wholesale_cost, ss_list_price@1 as ss_list_price, ss_coupon_amt@2 as ss_coupon_amt, d_year@3 as d_year, d_year@6 as d_year, d_year@7 as d_year, s_store_name@4 as s_store_name, s_zip@5 as s_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, ca_street_number@12 as ca_street_number, ca_street_name@13 as ca_street_name, ca_city@14 as ca_city, ca_zip@15 as ca_zip, i_item_sk@16 as i_item_sk, i_product_name@17 as i_product_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_wholesale_cost@1, ss_list_price@2, ss_coupon_amt@3, d_year@4, s_store_name@5, s_zip@6, d_year@7, d_year@8, ca_street_number@9, ca_street_name@10, ca_city@11, ca_zip@12, ca_street_number@13, ca_street_name@14, ca_city@15, ca_zip@16, i_item_sk@17, i_product_name@18] - │ CoalescePartitionsExec - │ [Stage 21] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 21] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ FilterExec: i_color@2 IN (SET) ([purple, burlywood, indian, spring, floral, medium]) AND i_current_price@1 >= Some(6500),4,2 AND i_current_price@1 <= Some(7400),4,2, projection=[i_item_sk@0, i_product_name@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_current_price, i_color, i_product_name], file_type=parquet, predicate=i_color@17 IN (SET) ([purple, burlywood, indian, spring, floral, medium]) AND i_current_price@5 >= Some(6500),4,2 AND i_current_price@5 <= Some(7400),4,2 AND DynamicFilter [ empty ], pruning_predicate=(i_color_null_count@2 != row_count@3 AND i_color_min@0 <= purple AND purple <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= burlywood AND burlywood <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= indian AND indian <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= spring AND spring <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= floral AND floral <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= medium AND medium <= i_color_max@1) AND i_current_price_null_count@5 != row_count@3 AND i_current_price_max@4 >= Some(6500),4,2 AND i_current_price_null_count@5 != row_count@3 AND i_current_price_min@6 <= Some(7400),4,2, required_guarantees=[i_color in (burlywood, floral, indian, medium, purple, spring)] └────────────────────────────────────────────────── - ┌───── Stage 21 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(hd_income_band_sk@9, ib_income_band_sk@0)], projection=[ss_item_sk@0, ss_wholesale_cost@1, ss_list_price@2, ss_coupon_amt@3, d_year@4, s_store_name@5, s_zip@6, d_year@7, d_year@8, ca_street_number@10, ca_street_name@11, ca_city@12, ca_zip@13, ca_street_number@14, ca_street_name@15, ca_city@16, ca_zip@17] - │ CoalescePartitionsExec - │ [Stage 20] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/income_band/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-3.parquet]]}, projection=[ib_income_band_sk], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 20 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ib_income_band_sk@0, hd_income_band_sk@9)], projection=[ss_item_sk@1, ss_wholesale_cost@2, ss_list_price@3, ss_coupon_amt@4, d_year@5, s_store_name@6, s_zip@7, d_year@8, d_year@9, hd_income_band_sk@11, ca_street_number@12, ca_street_name@13, ca_city@14, ca_zip@15, ca_street_number@16, ca_street_name@17, ca_city@18, ca_zip@19] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@7, ca_address_sk@0)], projection=[ss_item_sk@0, ss_wholesale_cost@1, ss_list_price@2, ss_coupon_amt@3, d_year@4, s_store_name@5, s_zip@6, d_year@8, d_year@9, hd_income_band_sk@10, hd_income_band_sk@11, ca_street_number@12, ca_street_name@13, ca_city@14, ca_zip@15, ca_street_number@17, ca_street_name@18, ca_city@19, ca_zip@20] - │ CoalescePartitionsExec - │ [Stage 19] => 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/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_street_number, ca_street_name, ca_city, ca_zip], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 21 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(hd_income_band_sk@9, ib_income_band_sk@0)], projection=[ss_item_sk@0, ss_wholesale_cost@1, ss_list_price@2, ss_coupon_amt@3, d_year@4, s_store_name@5, s_zip@6, d_year@7, d_year@8, ca_street_number@10, ca_street_name@11, ca_city@12, ca_zip@13, ca_street_number@14, ca_street_name@15, ca_city@16, ca_zip@17] + │ [Stage 20] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/income_band/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-3.parquet]]}, projection=[ib_income_band_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 20 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ib_income_band_sk@0, hd_income_band_sk@9)], projection=[ss_item_sk@1, ss_wholesale_cost@2, ss_list_price@3, ss_coupon_amt@4, d_year@5, s_store_name@6, s_zip@7, d_year@8, d_year@9, hd_income_band_sk@11, ca_street_number@12, ca_street_name@13, ca_city@14, ca_zip@15, ca_street_number@16, ca_street_name@17, ca_city@18, ca_zip@19] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@7, ca_address_sk@0)], projection=[ss_item_sk@0, ss_wholesale_cost@1, ss_list_price@2, ss_coupon_amt@3, d_year@4, s_store_name@5, s_zip@6, d_year@8, d_year@9, hd_income_band_sk@10, hd_income_band_sk@11, ca_street_number@12, ca_street_name@13, ca_city@14, ca_zip@15, ca_street_number@17, ca_street_name@18, ca_city@19, ca_zip@20] + │ [Stage 19] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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/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_street_number, ca_street_name, ca_city, ca_zip], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/income_band/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-3.parquet]]}, projection=[ib_income_band_sk], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/income_band/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-3.parquet]]}, projection=[ib_income_band_sk], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 19 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(ad1.ca_address_sk AS Float64)@5)], projection=[ss_item_sk@0, ss_wholesale_cost@2, ss_list_price@3, ss_coupon_amt@4, d_year@5, s_store_name@6, s_zip@7, c_current_addr_sk@8, d_year@9, d_year@10, hd_income_band_sk@11, hd_income_band_sk@12, ca_street_number@14, ca_street_name@15, ca_city@16, ca_zip@17] - │ CoalescePartitionsExec - │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_street_number@1 as ca_street_number, ca_street_name@2 as ca_street_name, ca_city@3 as ca_city, ca_zip@4 as ca_zip, CAST(ca_address_sk@0 AS Float64) as CAST(ad1.ca_address_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/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_street_number, ca_street_name, ca_city, ca_zip], file_type=parquet + ┌───── Stage 19 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(ad1.ca_address_sk AS Float64)@5)], projection=[ss_item_sk@0, ss_wholesale_cost@2, ss_list_price@3, ss_coupon_amt@4, d_year@5, s_store_name@6, s_zip@7, c_current_addr_sk@8, d_year@9, d_year@10, hd_income_band_sk@11, hd_income_band_sk@12, ca_street_number@14, ca_street_name@15, ca_city@16, ca_zip@17] + │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_street_number@1 as ca_street_number, ca_street_name@2 as ca_street_name, ca_city@3 as ca_city, ca_zip@4 as ca_zip, CAST(ca_address_sk@0 AS Float64) as CAST(ad1.ca_address_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/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_street_number, ca_street_name, ca_city, ca_zip], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 18 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_hdemo_sk@8, CAST(hd2.hd_demo_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_addr_sk@1, ss_wholesale_cost@2, ss_list_price@3, ss_coupon_amt@4, d_year@5, s_store_name@6, s_zip@7, c_current_addr_sk@9, d_year@10, d_year@11, hd_income_band_sk@12, hd_income_band_sk@14] - │ CoalescePartitionsExec - │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ 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_income_band_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(hd2.hd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 18 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_hdemo_sk@8, CAST(hd2.hd_demo_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_addr_sk@1, ss_wholesale_cost@2, ss_list_price@3, ss_coupon_amt@4, d_year@5, s_store_name@6, s_zip@7, c_current_addr_sk@9, d_year@10, d_year@11, hd_income_band_sk@12, hd_income_band_sk@14] + │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ 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_income_band_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(hd2.hd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 17 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_hdemo_sk@1, CAST(hd1.hd_demo_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_addr_sk@2, ss_wholesale_cost@3, ss_list_price@4, ss_coupon_amt@5, d_year@6, s_store_name@7, s_zip@8, c_current_hdemo_sk@9, c_current_addr_sk@10, d_year@11, d_year@12, hd_income_band_sk@14] - │ CoalescePartitionsExec - │ [Stage 16] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ 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_income_band_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(hd1.hd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 17 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_hdemo_sk@1, CAST(hd1.hd_demo_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_addr_sk@2, ss_wholesale_cost@3, ss_list_price@4, ss_coupon_amt@5, d_year@6, s_store_name@7, s_zip@8, c_current_hdemo_sk@9, c_current_addr_sk@10, d_year@11, d_year@12, hd_income_band_sk@14] + │ [Stage 16] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ 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_income_band_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(hd1.hd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(promotion.p_promo_sk AS Float64)@1, ss_promo_sk@3)], projection=[ss_item_sk@2, ss_hdemo_sk@3, ss_addr_sk@4, ss_wholesale_cost@6, ss_list_price@7, ss_coupon_amt@8, d_year@9, s_store_name@10, s_zip@11, c_current_hdemo_sk@12, c_current_addr_sk@13, d_year@14, d_year@15] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_cdemo_sk@10, CAST(cd2.cd_demo_sk AS Float64)@2)], filter=cd_marital_status@1 != cd_marital_status@0, projection=[ss_item_sk@0, ss_hdemo_sk@1, ss_addr_sk@2, ss_promo_sk@3, ss_wholesale_cost@4, ss_list_price@5, ss_coupon_amt@6, d_year@7, s_store_name@8, s_zip@9, c_current_hdemo_sk@11, c_current_addr_sk@12, d_year@13, d_year@14] - │ CoalescePartitionsExec - │ [Stage 15] => 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/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, CAST(cd_demo_sk@0 AS Float64) as CAST(cd2.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 16 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(promotion.p_promo_sk AS Float64)@1, ss_promo_sk@3)], projection=[ss_item_sk@2, ss_hdemo_sk@3, ss_addr_sk@4, ss_wholesale_cost@6, ss_list_price@7, ss_coupon_amt@8, d_year@9, s_store_name@10, s_zip@11, c_current_hdemo_sk@12, c_current_addr_sk@13, d_year@14, d_year@15] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_cdemo_sk@10, CAST(cd2.cd_demo_sk AS Float64)@2)], filter=cd_marital_status@1 != cd_marital_status@0, projection=[ss_item_sk@0, ss_hdemo_sk@1, ss_addr_sk@2, ss_promo_sk@3, ss_wholesale_cost@4, ss_list_price@5, ss_coupon_amt@6, d_year@7, s_store_name@8, s_zip@9, c_current_hdemo_sk@11, c_current_addr_sk@12, d_year@13, d_year@14] + │ [Stage 15] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, CAST(cd_demo_sk@0 AS Float64) as CAST(cd2.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)], file_type=parquet + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 15 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_cdemo_sk@1, CAST(cd1.cd_demo_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_hdemo_sk@2, ss_addr_sk@3, ss_promo_sk@4, ss_wholesale_cost@5, ss_list_price@6, ss_coupon_amt@7, d_year@8, s_store_name@9, s_zip@10, c_current_cdemo_sk@11, c_current_hdemo_sk@12, c_current_addr_sk@13, d_year@14, d_year@15, cd_marital_status@17] - │ CoalescePartitionsExec - │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, CAST(cd_demo_sk@0 AS Float64) as CAST(cd1.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 15 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_cdemo_sk@1, CAST(cd1.cd_demo_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_hdemo_sk@2, ss_addr_sk@3, ss_promo_sk@4, ss_wholesale_cost@5, ss_list_price@6, ss_coupon_amt@7, d_year@8, s_store_name@9, s_zip@10, c_current_cdemo_sk@11, c_current_hdemo_sk@12, c_current_addr_sk@13, d_year@14, d_year@15, cd_marital_status@17] + │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, CAST(cd_demo_sk@0 AS Float64) as CAST(cd1.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_first_shipto_date_sk@14, CAST(d3.d_date_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_cdemo_sk@1, ss_hdemo_sk@2, ss_addr_sk@3, ss_promo_sk@4, ss_wholesale_cost@5, ss_list_price@6, ss_coupon_amt@7, d_year@8, s_store_name@9, s_zip@10, c_current_cdemo_sk@11, c_current_hdemo_sk@12, c_current_addr_sk@13, d_year@15, d_year@17] - │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(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_year], file_type=parquet + ┌───── Stage 14 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_first_shipto_date_sk@14, CAST(d3.d_date_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_cdemo_sk@1, ss_hdemo_sk@2, ss_addr_sk@3, ss_promo_sk@4, ss_wholesale_cost@5, ss_list_price@6, ss_coupon_amt@7, d_year@8, s_store_name@9, s_zip@10, c_current_cdemo_sk@11, c_current_hdemo_sk@12, c_current_addr_sk@13, d_year@15, d_year@17] + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(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_year], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_first_sales_date_sk@15, CAST(d2.d_date_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_cdemo_sk@1, ss_hdemo_sk@2, ss_addr_sk@3, ss_promo_sk@4, ss_wholesale_cost@5, ss_list_price@6, ss_coupon_amt@7, d_year@8, s_store_name@9, s_zip@10, c_current_cdemo_sk@11, c_current_hdemo_sk@12, c_current_addr_sk@13, c_first_shipto_date_sk@14, d_year@17] - │ CoalescePartitionsExec - │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(d2.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_year], file_type=parquet + ┌───── Stage 13 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_first_sales_date_sk@15, CAST(d2.d_date_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_cdemo_sk@1, ss_hdemo_sk@2, ss_addr_sk@3, ss_promo_sk@4, ss_wholesale_cost@5, ss_list_price@6, ss_coupon_amt@7, d_year@8, s_store_name@9, s_zip@10, c_current_cdemo_sk@11, c_current_hdemo_sk@12, c_current_addr_sk@13, c_first_shipto_date_sk@14, d_year@17] + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(d2.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_year], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 12 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@6)], projection=[ss_item_sk@0, ss_cdemo_sk@2, ss_hdemo_sk@3, ss_addr_sk@4, ss_promo_sk@5, ss_wholesale_cost@6, ss_list_price@7, ss_coupon_amt@8, d_year@9, s_store_name@10, s_zip@11, c_current_cdemo_sk@13, c_current_hdemo_sk@14, c_current_addr_sk@15, c_first_shipto_date_sk@16, c_first_sales_date_sk@17] - │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_cdemo_sk@1 as c_current_cdemo_sk, c_current_hdemo_sk@2 as c_current_hdemo_sk, c_current_addr_sk@3 as c_current_addr_sk, c_first_shipto_date_sk@4 as c_first_shipto_date_sk, c_first_sales_date_sk@5 as c_first_sales_date_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_hdemo_sk, c_current_addr_sk, c_first_shipto_date_sk, c_first_sales_date_sk], file_type=parquet + ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@6)], projection=[ss_item_sk@0, ss_cdemo_sk@2, ss_hdemo_sk@3, ss_addr_sk@4, ss_promo_sk@5, ss_wholesale_cost@6, ss_list_price@7, ss_coupon_amt@8, d_year@9, s_store_name@10, s_zip@11, c_current_cdemo_sk@13, c_current_hdemo_sk@14, c_current_addr_sk@15, c_first_shipto_date_sk@16, c_first_sales_date_sk@17] + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_cdemo_sk@1 as c_current_cdemo_sk, c_current_hdemo_sk@2 as c_current_hdemo_sk, c_current_addr_sk@3 as c_current_addr_sk, c_first_shipto_date_sk@4 as c_first_shipto_date_sk, c_first_sales_date_sk@5 as c_first_sales_date_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_hdemo_sk, c_current_addr_sk, c_first_shipto_date_sk, c_first_sales_date_sk], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[ss_item_sk@2 as ss_item_sk, ss_customer_sk@3 as ss_customer_sk, ss_cdemo_sk@4 as ss_cdemo_sk, ss_hdemo_sk@5 as ss_hdemo_sk, ss_addr_sk@6 as ss_addr_sk, ss_promo_sk@7 as ss_promo_sk, ss_wholesale_cost@8 as ss_wholesale_cost, ss_list_price@9 as ss_list_price, ss_coupon_amt@10 as ss_coupon_amt, d_year@11 as d_year, s_store_name@0 as s_store_name, s_zip@1 as s_zip] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@5)], projection=[s_store_name@1, s_zip@2, ss_item_sk@4, ss_customer_sk@5, ss_cdemo_sk@6, ss_hdemo_sk@7, ss_addr_sk@8, ss_promo_sk@10, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13, d_year@14] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_customer_sk@2 as ss_customer_sk, ss_cdemo_sk@3 as ss_cdemo_sk, ss_hdemo_sk@4 as ss_hdemo_sk, ss_addr_sk@5 as ss_addr_sk, ss_store_sk@6 as ss_store_sk, ss_promo_sk@7 as ss_promo_sk, ss_wholesale_cost@8 as ss_wholesale_cost, ss_list_price@9 as ss_list_price, ss_coupon_amt@10 as ss_coupon_amt, d_year@0 as d_year] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_customer_sk@5, ss_cdemo_sk@6, ss_hdemo_sk@7, ss_addr_sk@8, ss_store_sk@9, ss_promo_sk@10, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, ss_item_sk@1)], projection=[ss_sold_date_sk@1, ss_item_sk@2, ss_customer_sk@3, ss_cdemo_sk@4, ss_hdemo_sk@5, ss_addr_sk@6, ss_store_sk@7, ss_promo_sk@8, ss_wholesale_cost@9, ss_list_price@10, ss_coupon_amt@11] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_item_sk@0, ss_item_sk@1), (sr_ticket_number@1, ss_ticket_number@8)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_customer_sk@4, ss_cdemo_sk@5, ss_hdemo_sk@6, ss_addr_sk@7, ss_store_sk@8, ss_promo_sk@9, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13] - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ss_item_sk@2 as ss_item_sk, ss_customer_sk@3 as ss_customer_sk, ss_cdemo_sk@4 as ss_cdemo_sk, ss_hdemo_sk@5 as ss_hdemo_sk, ss_addr_sk@6 as ss_addr_sk, ss_promo_sk@7 as ss_promo_sk, ss_wholesale_cost@8 as ss_wholesale_cost, ss_list_price@9 as ss_list_price, ss_coupon_amt@10 as ss_coupon_amt, d_year@11 as d_year, s_store_name@0 as s_store_name, s_zip@1 as s_zip] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@5)], projection=[s_store_name@1, s_zip@2, ss_item_sk@4, ss_customer_sk@5, ss_cdemo_sk@6, ss_hdemo_sk@7, ss_addr_sk@8, ss_promo_sk@10, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13, d_year@14] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_customer_sk@2 as ss_customer_sk, ss_cdemo_sk@3 as ss_cdemo_sk, ss_hdemo_sk@4 as ss_hdemo_sk, ss_addr_sk@5 as ss_addr_sk, ss_store_sk@6 as ss_store_sk, ss_promo_sk@7 as ss_promo_sk, ss_wholesale_cost@8 as ss_wholesale_cost, ss_list_price@9 as ss_list_price, ss_coupon_amt@10 as ss_coupon_amt, d_year@0 as d_year] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_customer_sk@5, ss_cdemo_sk@6, ss_hdemo_sk@7, ss_addr_sk@8, ss_store_sk@9, ss_promo_sk@10, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13] + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, ss_item_sk@1)], projection=[ss_sold_date_sk@1, ss_item_sk@2, ss_customer_sk@3, ss_cdemo_sk@4, ss_hdemo_sk@5, ss_addr_sk@6, ss_store_sk@7, ss_promo_sk@8, ss_wholesale_cost@9, ss_list_price@10, ss_coupon_amt@11] + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_item_sk@0, ss_item_sk@1), (sr_ticket_number@1, ss_ticket_number@8)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_customer_sk@4, ss_cdemo_sk@5, ss_hdemo_sk@6, ss_addr_sk@7, ss_store_sk@8, ss_promo_sk@9, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13] + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 1999 - │ 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_year], file_type=parquet, predicate=d_year@6 = 1999, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1, required_guarantees=[d_year in (1999)] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 1999 + │ 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_year], file_type=parquet, predicate=d_year@6 = 1999, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1, required_guarantees=[d_year in (1999)] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: CAST(sum(catalog_sales.cs_ext_list_price)@1 AS Decimal128(38, 2)) > Some(2),20,0 * sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)@2, projection=[cs_item_sk@0] - │ AggregateExec: mode=FinalPartitioned, gby=[cs_item_sk@0 as cs_item_sk], aggr=[sum(catalog_sales.cs_ext_list_price), sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)] - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: CAST(sum(catalog_sales.cs_ext_list_price)@1 AS Decimal128(38, 2)) > Some(2),20,0 * sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)@2, projection=[cs_item_sk@0] + │ AggregateExec: mode=FinalPartitioned, gby=[cs_item_sk@0 as cs_item_sk], aggr=[sum(catalog_sales.cs_ext_list_price), sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)] + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([cs_item_sk@0], 6), input_partitions=3 @@ -6867,153 +6884,153 @@ mod tests { │ AggregateExec: mode=Partial, gby=[i_product_name@17 as i_product_name, i_item_sk@16 as i_item_sk, s_store_name@6 as s_store_name, s_zip@7 as s_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, ca_street_number@12 as ca_street_number, ca_street_name@13 as ca_street_name, ca_city@14 as ca_city, ca_zip@15 as ca_zip, d_year@3 as d_year, d_year@4 as d_year, d_year@5 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]) │ ProjectionExec: expr=[ss_wholesale_cost@0 as ss_wholesale_cost, ss_list_price@1 as ss_list_price, ss_coupon_amt@2 as ss_coupon_amt, d_year@3 as d_year, d_year@6 as d_year, d_year@7 as d_year, s_store_name@4 as s_store_name, s_zip@5 as s_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, ca_street_number@12 as ca_street_number, ca_street_name@13 as ca_street_name, ca_city@14 as ca_city, ca_zip@15 as ca_zip, i_item_sk@16 as i_item_sk, i_product_name@17 as i_product_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_wholesale_cost@1, ss_list_price@2, ss_coupon_amt@3, d_year@4, s_store_name@5, s_zip@6, d_year@7, d_year@8, ca_street_number@9, ca_street_name@10, ca_city@11, ca_zip@12, ca_street_number@13, ca_street_name@14, ca_city@15, ca_zip@16, i_item_sk@17, i_product_name@18] - │ CoalescePartitionsExec - │ [Stage 44] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 44] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ FilterExec: i_color@2 IN (SET) ([purple, burlywood, indian, spring, floral, medium]) AND i_current_price@1 >= Some(6500),4,2 AND i_current_price@1 <= Some(7400),4,2, projection=[i_item_sk@0, i_product_name@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_current_price, i_color, i_product_name], file_type=parquet, predicate=i_color@17 IN (SET) ([purple, burlywood, indian, spring, floral, medium]) AND i_current_price@5 >= Some(6500),4,2 AND i_current_price@5 <= Some(7400),4,2 AND DynamicFilter [ empty ], pruning_predicate=(i_color_null_count@2 != row_count@3 AND i_color_min@0 <= purple AND purple <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= burlywood AND burlywood <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= indian AND indian <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= spring AND spring <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= floral AND floral <= i_color_max@1 OR i_color_null_count@2 != row_count@3 AND i_color_min@0 <= medium AND medium <= i_color_max@1) AND i_current_price_null_count@5 != row_count@3 AND i_current_price_max@4 >= Some(6500),4,2 AND i_current_price_null_count@5 != row_count@3 AND i_current_price_min@6 <= Some(7400),4,2, required_guarantees=[i_color in (burlywood, floral, indian, medium, purple, spring)] └────────────────────────────────────────────────── - ┌───── Stage 44 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(hd_income_band_sk@9, ib_income_band_sk@0)], projection=[ss_item_sk@0, ss_wholesale_cost@1, ss_list_price@2, ss_coupon_amt@3, d_year@4, s_store_name@5, s_zip@6, d_year@7, d_year@8, ca_street_number@10, ca_street_name@11, ca_city@12, ca_zip@13, ca_street_number@14, ca_street_name@15, ca_city@16, ca_zip@17] - │ CoalescePartitionsExec - │ [Stage 43] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/income_band/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-3.parquet]]}, projection=[ib_income_band_sk], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 43 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ib_income_band_sk@0, hd_income_band_sk@9)], projection=[ss_item_sk@1, ss_wholesale_cost@2, ss_list_price@3, ss_coupon_amt@4, d_year@5, s_store_name@6, s_zip@7, d_year@8, d_year@9, hd_income_band_sk@11, ca_street_number@12, ca_street_name@13, ca_city@14, ca_zip@15, ca_street_number@16, ca_street_name@17, ca_city@18, ca_zip@19] - │ CoalescePartitionsExec - │ [Stage 24] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@7, ca_address_sk@0)], projection=[ss_item_sk@0, ss_wholesale_cost@1, ss_list_price@2, ss_coupon_amt@3, d_year@4, s_store_name@5, s_zip@6, d_year@8, d_year@9, hd_income_band_sk@10, hd_income_band_sk@11, ca_street_number@12, ca_street_name@13, ca_city@14, ca_zip@15, ca_street_number@17, ca_street_name@18, ca_city@19, ca_zip@20] - │ CoalescePartitionsExec - │ [Stage 42] => 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/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_street_number, ca_street_name, ca_city, ca_zip], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 44 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(hd_income_band_sk@9, ib_income_band_sk@0)], projection=[ss_item_sk@0, ss_wholesale_cost@1, ss_list_price@2, ss_coupon_amt@3, d_year@4, s_store_name@5, s_zip@6, d_year@7, d_year@8, ca_street_number@10, ca_street_name@11, ca_city@12, ca_zip@13, ca_street_number@14, ca_street_name@15, ca_city@16, ca_zip@17] + │ [Stage 43] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/income_band/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-3.parquet]]}, projection=[ib_income_band_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 43 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ib_income_band_sk@0, hd_income_band_sk@9)], projection=[ss_item_sk@1, ss_wholesale_cost@2, ss_list_price@3, ss_coupon_amt@4, d_year@5, s_store_name@6, s_zip@7, d_year@8, d_year@9, hd_income_band_sk@11, ca_street_number@12, ca_street_name@13, ca_city@14, ca_zip@15, ca_street_number@16, ca_street_name@17, ca_city@18, ca_zip@19] + │ [Stage 24] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@7, ca_address_sk@0)], projection=[ss_item_sk@0, ss_wholesale_cost@1, ss_list_price@2, ss_coupon_amt@3, d_year@4, s_store_name@5, s_zip@6, d_year@8, d_year@9, hd_income_band_sk@10, hd_income_band_sk@11, ca_street_number@12, ca_street_name@13, ca_city@14, ca_zip@15, ca_street_number@17, ca_street_name@18, ca_city@19, ca_zip@20] + │ [Stage 42] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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/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_street_number, ca_street_name, ca_city, ca_zip], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 24 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/income_band/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-3.parquet]]}, projection=[ib_income_band_sk], file_type=parquet + ┌───── Stage 24 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/income_band/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-3.parquet]]}, projection=[ib_income_band_sk], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 42 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(ad1.ca_address_sk AS Float64)@5)], projection=[ss_item_sk@0, ss_wholesale_cost@2, ss_list_price@3, ss_coupon_amt@4, d_year@5, s_store_name@6, s_zip@7, c_current_addr_sk@8, d_year@9, d_year@10, hd_income_band_sk@11, hd_income_band_sk@12, ca_street_number@14, ca_street_name@15, ca_city@16, ca_zip@17] - │ CoalescePartitionsExec - │ [Stage 41] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_street_number@1 as ca_street_number, ca_street_name@2 as ca_street_name, ca_city@3 as ca_city, ca_zip@4 as ca_zip, CAST(ca_address_sk@0 AS Float64) as CAST(ad1.ca_address_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/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_street_number, ca_street_name, ca_city, ca_zip], file_type=parquet + ┌───── Stage 42 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(ad1.ca_address_sk AS Float64)@5)], projection=[ss_item_sk@0, ss_wholesale_cost@2, ss_list_price@3, ss_coupon_amt@4, d_year@5, s_store_name@6, s_zip@7, c_current_addr_sk@8, d_year@9, d_year@10, hd_income_band_sk@11, hd_income_band_sk@12, ca_street_number@14, ca_street_name@15, ca_city@16, ca_zip@17] + │ [Stage 41] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_street_number@1 as ca_street_number, ca_street_name@2 as ca_street_name, ca_city@3 as ca_city, ca_zip@4 as ca_zip, CAST(ca_address_sk@0 AS Float64) as CAST(ad1.ca_address_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/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_street_number, ca_street_name, ca_city, ca_zip], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 41 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_hdemo_sk@8, CAST(hd2.hd_demo_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_addr_sk@1, ss_wholesale_cost@2, ss_list_price@3, ss_coupon_amt@4, d_year@5, s_store_name@6, s_zip@7, c_current_addr_sk@9, d_year@10, d_year@11, hd_income_band_sk@12, hd_income_band_sk@14] - │ CoalescePartitionsExec - │ [Stage 40] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ 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_income_band_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(hd2.hd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 41 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_hdemo_sk@8, CAST(hd2.hd_demo_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_addr_sk@1, ss_wholesale_cost@2, ss_list_price@3, ss_coupon_amt@4, d_year@5, s_store_name@6, s_zip@7, c_current_addr_sk@9, d_year@10, d_year@11, hd_income_band_sk@12, hd_income_band_sk@14] + │ [Stage 40] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ 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_income_band_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(hd2.hd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 40 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_hdemo_sk@1, CAST(hd1.hd_demo_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_addr_sk@2, ss_wholesale_cost@3, ss_list_price@4, ss_coupon_amt@5, d_year@6, s_store_name@7, s_zip@8, c_current_hdemo_sk@9, c_current_addr_sk@10, d_year@11, d_year@12, hd_income_band_sk@14] - │ CoalescePartitionsExec - │ [Stage 39] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ 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_income_band_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(hd1.hd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 40 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_hdemo_sk@1, CAST(hd1.hd_demo_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_addr_sk@2, ss_wholesale_cost@3, ss_list_price@4, ss_coupon_amt@5, d_year@6, s_store_name@7, s_zip@8, c_current_hdemo_sk@9, c_current_addr_sk@10, d_year@11, d_year@12, hd_income_band_sk@14] + │ [Stage 39] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ 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_income_band_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(hd1.hd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 39 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(promotion.p_promo_sk AS Float64)@1, ss_promo_sk@3)], projection=[ss_item_sk@2, ss_hdemo_sk@3, ss_addr_sk@4, ss_wholesale_cost@6, ss_list_price@7, ss_coupon_amt@8, d_year@9, s_store_name@10, s_zip@11, c_current_hdemo_sk@12, c_current_addr_sk@13, d_year@14, d_year@15] - │ CoalescePartitionsExec - │ [Stage 25] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_cdemo_sk@10, CAST(cd2.cd_demo_sk AS Float64)@2)], filter=cd_marital_status@1 != cd_marital_status@0, projection=[ss_item_sk@0, ss_hdemo_sk@1, ss_addr_sk@2, ss_promo_sk@3, ss_wholesale_cost@4, ss_list_price@5, ss_coupon_amt@6, d_year@7, s_store_name@8, s_zip@9, c_current_hdemo_sk@11, c_current_addr_sk@12, d_year@13, d_year@14] - │ CoalescePartitionsExec - │ [Stage 38] => 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/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, CAST(cd_demo_sk@0 AS Float64) as CAST(cd2.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 39 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(promotion.p_promo_sk AS Float64)@1, ss_promo_sk@3)], projection=[ss_item_sk@2, ss_hdemo_sk@3, ss_addr_sk@4, ss_wholesale_cost@6, ss_list_price@7, ss_coupon_amt@8, d_year@9, s_store_name@10, s_zip@11, c_current_hdemo_sk@12, c_current_addr_sk@13, d_year@14, d_year@15] + │ [Stage 25] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_cdemo_sk@10, CAST(cd2.cd_demo_sk AS Float64)@2)], filter=cd_marital_status@1 != cd_marital_status@0, projection=[ss_item_sk@0, ss_hdemo_sk@1, ss_addr_sk@2, ss_promo_sk@3, ss_wholesale_cost@4, ss_list_price@5, ss_coupon_amt@6, d_year@7, s_store_name@8, s_zip@9, c_current_hdemo_sk@11, c_current_addr_sk@12, d_year@13, d_year@14] + │ [Stage 38] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, CAST(cd_demo_sk@0 AS Float64) as CAST(cd2.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 25 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)], file_type=parquet + ┌───── Stage 25 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 38 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_cdemo_sk@1, CAST(cd1.cd_demo_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_hdemo_sk@2, ss_addr_sk@3, ss_promo_sk@4, ss_wholesale_cost@5, ss_list_price@6, ss_coupon_amt@7, d_year@8, s_store_name@9, s_zip@10, c_current_cdemo_sk@11, c_current_hdemo_sk@12, c_current_addr_sk@13, d_year@14, d_year@15, cd_marital_status@17] - │ CoalescePartitionsExec - │ [Stage 37] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, CAST(cd_demo_sk@0 AS Float64) as CAST(cd1.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 38 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_cdemo_sk@1, CAST(cd1.cd_demo_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_hdemo_sk@2, ss_addr_sk@3, ss_promo_sk@4, ss_wholesale_cost@5, ss_list_price@6, ss_coupon_amt@7, d_year@8, s_store_name@9, s_zip@10, c_current_cdemo_sk@11, c_current_hdemo_sk@12, c_current_addr_sk@13, d_year@14, d_year@15, cd_marital_status@17] + │ [Stage 37] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, CAST(cd_demo_sk@0 AS Float64) as CAST(cd1.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 37 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_first_shipto_date_sk@14, CAST(d3.d_date_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_cdemo_sk@1, ss_hdemo_sk@2, ss_addr_sk@3, ss_promo_sk@4, ss_wholesale_cost@5, ss_list_price@6, ss_coupon_amt@7, d_year@8, s_store_name@9, s_zip@10, c_current_cdemo_sk@11, c_current_hdemo_sk@12, c_current_addr_sk@13, d_year@15, d_year@17] - │ CoalescePartitionsExec - │ [Stage 36] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(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_year], file_type=parquet + ┌───── Stage 37 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_first_shipto_date_sk@14, CAST(d3.d_date_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_cdemo_sk@1, ss_hdemo_sk@2, ss_addr_sk@3, ss_promo_sk@4, ss_wholesale_cost@5, ss_list_price@6, ss_coupon_amt@7, d_year@8, s_store_name@9, s_zip@10, c_current_cdemo_sk@11, c_current_hdemo_sk@12, c_current_addr_sk@13, d_year@15, d_year@17] + │ [Stage 36] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(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_year], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 36 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_first_sales_date_sk@15, CAST(d2.d_date_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_cdemo_sk@1, ss_hdemo_sk@2, ss_addr_sk@3, ss_promo_sk@4, ss_wholesale_cost@5, ss_list_price@6, ss_coupon_amt@7, d_year@8, s_store_name@9, s_zip@10, c_current_cdemo_sk@11, c_current_hdemo_sk@12, c_current_addr_sk@13, c_first_shipto_date_sk@14, d_year@17] - │ CoalescePartitionsExec - │ [Stage 35] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(d2.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_year], file_type=parquet + ┌───── Stage 36 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_first_sales_date_sk@15, CAST(d2.d_date_sk AS Float64)@2)], projection=[ss_item_sk@0, ss_cdemo_sk@1, ss_hdemo_sk@2, ss_addr_sk@3, ss_promo_sk@4, ss_wholesale_cost@5, ss_list_price@6, ss_coupon_amt@7, d_year@8, s_store_name@9, s_zip@10, c_current_cdemo_sk@11, c_current_hdemo_sk@12, c_current_addr_sk@13, c_first_shipto_date_sk@14, d_year@17] + │ [Stage 35] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(d2.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_year], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 35 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@6)], projection=[ss_item_sk@0, ss_cdemo_sk@2, ss_hdemo_sk@3, ss_addr_sk@4, ss_promo_sk@5, ss_wholesale_cost@6, ss_list_price@7, ss_coupon_amt@8, d_year@9, s_store_name@10, s_zip@11, c_current_cdemo_sk@13, c_current_hdemo_sk@14, c_current_addr_sk@15, c_first_shipto_date_sk@16, c_first_sales_date_sk@17] - │ CoalescePartitionsExec - │ [Stage 34] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_cdemo_sk@1 as c_current_cdemo_sk, c_current_hdemo_sk@2 as c_current_hdemo_sk, c_current_addr_sk@3 as c_current_addr_sk, c_first_shipto_date_sk@4 as c_first_shipto_date_sk, c_first_sales_date_sk@5 as c_first_sales_date_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_hdemo_sk, c_current_addr_sk, c_first_shipto_date_sk, c_first_sales_date_sk], file_type=parquet + ┌───── Stage 35 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@6)], projection=[ss_item_sk@0, ss_cdemo_sk@2, ss_hdemo_sk@3, ss_addr_sk@4, ss_promo_sk@5, ss_wholesale_cost@6, ss_list_price@7, ss_coupon_amt@8, d_year@9, s_store_name@10, s_zip@11, c_current_cdemo_sk@13, c_current_hdemo_sk@14, c_current_addr_sk@15, c_first_shipto_date_sk@16, c_first_sales_date_sk@17] + │ [Stage 34] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_cdemo_sk@1 as c_current_cdemo_sk, c_current_hdemo_sk@2 as c_current_hdemo_sk, c_current_addr_sk@3 as c_current_addr_sk, c_first_shipto_date_sk@4 as c_first_shipto_date_sk, c_first_sales_date_sk@5 as c_first_sales_date_sk, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_hdemo_sk, c_current_addr_sk, c_first_shipto_date_sk, c_first_sales_date_sk], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 34 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[ss_item_sk@2 as ss_item_sk, ss_customer_sk@3 as ss_customer_sk, ss_cdemo_sk@4 as ss_cdemo_sk, ss_hdemo_sk@5 as ss_hdemo_sk, ss_addr_sk@6 as ss_addr_sk, ss_promo_sk@7 as ss_promo_sk, ss_wholesale_cost@8 as ss_wholesale_cost, ss_list_price@9 as ss_list_price, ss_coupon_amt@10 as ss_coupon_amt, d_year@11 as d_year, s_store_name@0 as s_store_name, s_zip@1 as s_zip] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@5)], projection=[s_store_name@1, s_zip@2, ss_item_sk@4, ss_customer_sk@5, ss_cdemo_sk@6, ss_hdemo_sk@7, ss_addr_sk@8, ss_promo_sk@10, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13, d_year@14] - │ CoalescePartitionsExec - │ [Stage 26] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_customer_sk@2 as ss_customer_sk, ss_cdemo_sk@3 as ss_cdemo_sk, ss_hdemo_sk@4 as ss_hdemo_sk, ss_addr_sk@5 as ss_addr_sk, ss_store_sk@6 as ss_store_sk, ss_promo_sk@7 as ss_promo_sk, ss_wholesale_cost@8 as ss_wholesale_cost, ss_list_price@9 as ss_list_price, ss_coupon_amt@10 as ss_coupon_amt, d_year@0 as d_year] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_customer_sk@5, ss_cdemo_sk@6, ss_hdemo_sk@7, ss_addr_sk@8, ss_store_sk@9, ss_promo_sk@10, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13] - │ CoalescePartitionsExec - │ [Stage 27] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, ss_item_sk@1)], projection=[ss_sold_date_sk@1, ss_item_sk@2, ss_customer_sk@3, ss_cdemo_sk@4, ss_hdemo_sk@5, ss_addr_sk@6, ss_store_sk@7, ss_promo_sk@8, ss_wholesale_cost@9, ss_list_price@10, ss_coupon_amt@11] - │ CoalescePartitionsExec - │ [Stage 31] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_item_sk@0, ss_item_sk@1), (sr_ticket_number@1, ss_ticket_number@8)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_customer_sk@4, ss_cdemo_sk@5, ss_hdemo_sk@6, ss_addr_sk@7, ss_store_sk@8, ss_promo_sk@9, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13] - │ [Stage 32] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 33] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 34 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ss_item_sk@2 as ss_item_sk, ss_customer_sk@3 as ss_customer_sk, ss_cdemo_sk@4 as ss_cdemo_sk, ss_hdemo_sk@5 as ss_hdemo_sk, ss_addr_sk@6 as ss_addr_sk, ss_promo_sk@7 as ss_promo_sk, ss_wholesale_cost@8 as ss_wholesale_cost, ss_list_price@9 as ss_list_price, ss_coupon_amt@10 as ss_coupon_amt, d_year@11 as d_year, s_store_name@0 as s_store_name, s_zip@1 as s_zip] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@5)], projection=[s_store_name@1, s_zip@2, ss_item_sk@4, ss_customer_sk@5, ss_cdemo_sk@6, ss_hdemo_sk@7, ss_addr_sk@8, ss_promo_sk@10, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13, d_year@14] + │ [Stage 26] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_customer_sk@2 as ss_customer_sk, ss_cdemo_sk@3 as ss_cdemo_sk, ss_hdemo_sk@4 as ss_hdemo_sk, ss_addr_sk@5 as ss_addr_sk, ss_store_sk@6 as ss_store_sk, ss_promo_sk@7 as ss_promo_sk, ss_wholesale_cost@8 as ss_wholesale_cost, ss_list_price@9 as ss_list_price, ss_coupon_amt@10 as ss_coupon_amt, d_year@0 as d_year] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_year@1, ss_item_sk@4, ss_customer_sk@5, ss_cdemo_sk@6, ss_hdemo_sk@7, ss_addr_sk@8, ss_store_sk@9, ss_promo_sk@10, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13] + │ [Stage 27] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_item_sk@0, ss_item_sk@1)], projection=[ss_sold_date_sk@1, ss_item_sk@2, ss_customer_sk@3, ss_cdemo_sk@4, ss_hdemo_sk@5, ss_addr_sk@6, ss_store_sk@7, ss_promo_sk@8, ss_wholesale_cost@9, ss_list_price@10, ss_coupon_amt@11] + │ [Stage 31] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(sr_item_sk@0, ss_item_sk@1), (sr_ticket_number@1, ss_ticket_number@8)], projection=[ss_sold_date_sk@2, ss_item_sk@3, ss_customer_sk@4, ss_cdemo_sk@5, ss_hdemo_sk@6, ss_addr_sk@7, ss_store_sk@8, ss_promo_sk@9, ss_wholesale_cost@11, ss_list_price@12, ss_coupon_amt@13] + │ [Stage 32] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 33] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 26 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + ┌───── Stage 26 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_zip, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 27 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2000 - │ 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_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 27 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2000 + │ 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_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 31 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: CAST(sum(catalog_sales.cs_ext_list_price)@1 AS Decimal128(38, 2)) > Some(2),20,0 * sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)@2, projection=[cs_item_sk@0] - │ AggregateExec: mode=FinalPartitioned, gby=[cs_item_sk@0 as cs_item_sk], aggr=[sum(catalog_sales.cs_ext_list_price), sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)] - │ [Stage 30] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 31 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: CAST(sum(catalog_sales.cs_ext_list_price)@1 AS Decimal128(38, 2)) > Some(2),20,0 * sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)@2, projection=[cs_item_sk@0] + │ AggregateExec: mode=FinalPartitioned, gby=[cs_item_sk@0 as cs_item_sk], aggr=[sum(catalog_sales.cs_ext_list_price), sum(catalog_returns.cr_refunded_cash + catalog_returns.cr_reversed_charge + catalog_returns.cr_store_credit)] + │ [Stage 30] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 30 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([cs_item_sk@0], 6), input_partitions=3 @@ -7058,51 +7075,51 @@ mod tests { │ 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 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@2, i_item_sk@0)], projection=[s_store_name@0, ss_store_sk@1, revenue@3, i_item_desc@5, i_current_price@6, i_wholesale_cost@7, i_brand@8] - │ CoalescePartitionsExec - │ [Stage 4] => 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/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_desc, i_current_price, i_wholesale_cost, i_brand], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, ss_store_sk@0)], projection=[s_store_name@1, ss_store_sk@3, ss_item_sk@4, revenue@5] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=2 - │ ProjectionExec: expr=[ss_store_sk@0 as ss_store_sk, ss_item_sk@1 as ss_item_sk, sum(store_sales.ss_sales_price)@2 as revenue] - │ AggregateExec: mode=FinalPartitioned, gby=[ss_store_sk@0 as ss_store_sk, ss_item_sk@1 as ss_item_sk], aggr=[sum(store_sales.ss_sales_price)] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 5 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@2, i_item_sk@0)], projection=[s_store_name@0, ss_store_sk@1, revenue@3, i_item_desc@5, i_current_price@6, i_wholesale_cost@7, i_brand@8] + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_desc, i_current_price, i_wholesale_cost, i_brand], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, ss_store_sk@0)], projection=[s_store_name@1, ss_store_sk@3, ss_item_sk@4, revenue@5] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, input_tasks=2 + │ ProjectionExec: expr=[ss_store_sk@0 as ss_store_sk, ss_item_sk@1 as ss_item_sk, sum(store_sales.ss_sales_price)@2 as revenue] + │ AggregateExec: mode=FinalPartitioned, gby=[ss_store_sk@0 as ss_store_sk, ss_item_sk@1 as ss_item_sk], aggr=[sum(store_sales.ss_sales_price)] + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] - │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([ss_store_sk@0, ss_item_sk@1], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[ss_store_sk@1 as ss_store_sk, ss_item_sk@0 as ss_item_sk], aggr=[sum(store_sales.ss_sales_price)] │ 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_store_sk@4, ss_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_month_seq@1 >= 1176 AND d_month_seq@1 <= 1187, projection=[d_date_sk@0] - │ 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_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1176 AND d_month_seq@3 <= 1187, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1176 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1187, required_guarantees=[] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_month_seq@1 >= 1176 AND d_month_seq@1 <= 1187, projection=[d_date_sk@0] + │ 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_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1176 AND d_month_seq@3 <= 1187, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1176 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1187, required_guarantees=[] └────────────────────────────────────────────────── ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p0..p5] │ RepartitionExec: partitioning=Hash([ss_store_sk@0], 6), input_partitions=3 @@ -7115,18 +7132,18 @@ mod tests { │ RepartitionExec: partitioning=Hash([ss_store_sk@0, ss_item_sk@1], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[ss_store_sk@1 as ss_store_sk, ss_item_sk@0 as ss_item_sk], aggr=[sum(store_sales.ss_sales_price)] │ 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_store_sk@4, ss_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_month_seq@1 >= 1176 AND d_month_seq@1 <= 1187, projection=[d_date_sk@0] - │ 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_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1176 AND d_month_seq@3 <= 1187, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1176 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1187, required_guarantees=[] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_month_seq@1 >= 1176 AND d_month_seq@1 <= 1187, projection=[d_date_sk@0] + │ 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_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1176 AND d_month_seq@3 <= 1187, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1176 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1187, required_guarantees=[] └────────────────────────────────────────────────── "); Ok(()) @@ -7159,98 +7176,98 @@ mod tests { │ AggregateExec: mode=Partial, gby=[w_warehouse_name@15 as w_warehouse_name, w_warehouse_sq_ft@16 as w_warehouse_sq_ft, w_city@17 as w_city, w_county@18 as w_county, w_state@19 as w_state, w_country@20 as w_country, d_year@21 as d_year], aggr=[sum(CASE WHEN date_dim.d_moy = Int64(1) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(2) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(3) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(4) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(5) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(6) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(7) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(8) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(9) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(10) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(11) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(12) THEN web_sales.ws_ext_sales_price * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(1) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(2) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(3) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(4) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(5) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(6) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(7) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(8) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(9) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(10) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(11) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(12) THEN web_sales.ws_net_paid * web_sales.ws_quantity ELSE Int64(0) END)], ordering_mode=PartiallySorted([6]) │ ProjectionExec: expr=[d_moy@10 = 1 as __common_expr_2, d_moy@10 = 2 as __common_expr_3, d_moy@10 = 3 as __common_expr_4, d_moy@10 = 4 as __common_expr_5, d_moy@10 = 5 as __common_expr_6, d_moy@10 = 6 as __common_expr_7, d_moy@10 = 7 as __common_expr_8, d_moy@10 = 8 as __common_expr_9, d_moy@10 = 9 as __common_expr_10, d_moy@10 = 10 as __common_expr_11, d_moy@10 = 11 as __common_expr_12, d_moy@10 = 12 as __common_expr_13, ws_quantity@0 as ws_quantity, ws_ext_sales_price@1 as ws_ext_sales_price, ws_net_paid@2 as ws_net_paid, w_warehouse_name@3 as w_warehouse_name, w_warehouse_sq_ft@4 as w_warehouse_sq_ft, w_city@5 as w_city, w_county@6 as w_county, w_state@7 as w_state, w_country@8 as w_country, d_year@9 as d_year] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(ship_mode.sm_ship_mode_sk AS Float64)@1, ws_ship_mode_sk@0)], projection=[ws_quantity@3, ws_ext_sales_price@4, ws_net_paid@5, w_warehouse_name@6, w_warehouse_sq_ft@7, w_city@8, w_county@9, w_state@10, w_country@11, d_year@12, d_moy@13] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_sold_time_sk@0, CAST(time_dim.t_time_sk AS Float64)@1)], projection=[ws_ship_mode_sk@1, ws_quantity@2, ws_ext_sales_price@3, ws_net_paid@4, w_warehouse_name@5, w_warehouse_sq_ft@6, w_city@7, w_county@8, w_state@9, w_country@10, d_year@11, d_moy@12] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] │ FilterExec: t_time@1 >= 30838 AND t_time@1 <= 59638, projection=[t_time_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/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_time], file_type=parquet, predicate=t_time@2 >= 30838 AND t_time@2 <= 59638, pruning_predicate=t_time_null_count@1 != row_count@2 AND t_time_max@0 >= 30838 AND t_time_null_count@1 != row_count@2 AND t_time_min@3 <= 59638, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[sm_ship_mode_sk@0 as sm_ship_mode_sk, CAST(sm_ship_mode_sk@0 AS Float64) as CAST(ship_mode.sm_ship_mode_sk AS Float64)] - │ FilterExec: sm_carrier@1 = DHL OR sm_carrier@1 = BARIAN, projection=[sm_ship_mode_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-3.parquet]]}, projection=[sm_ship_mode_sk, sm_carrier], file_type=parquet, predicate=sm_carrier@4 = DHL OR sm_carrier@4 = BARIAN, pruning_predicate=sm_carrier_null_count@2 != row_count@3 AND sm_carrier_min@0 <= DHL AND DHL <= sm_carrier_max@1 OR sm_carrier_null_count@2 != row_count@3 AND sm_carrier_min@0 <= BARIAN AND BARIAN <= sm_carrier_max@1, required_guarantees=[sm_carrier in (BARIAN, DHL)] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@3)], projection=[ws_sold_time_sk@1, ws_ship_mode_sk@2, ws_quantity@3, ws_ext_sales_price@4, ws_net_paid@5, w_warehouse_name@6, w_warehouse_sq_ft@7, w_city@8, w_county@9, w_state@10, w_country@11, d_year@13, d_moy@14] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001 + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[sm_ship_mode_sk@0 as sm_ship_mode_sk, CAST(sm_ship_mode_sk@0 AS Float64) as CAST(ship_mode.sm_ship_mode_sk AS Float64)] + │ FilterExec: sm_carrier@1 = DHL OR sm_carrier@1 = BARIAN, projection=[sm_ship_mode_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, d_moy], 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 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ ProjectionExec: expr=[ws_sold_date_sk@6 as ws_sold_date_sk, ws_sold_time_sk@7 as ws_sold_time_sk, ws_ship_mode_sk@8 as ws_ship_mode_sk, ws_quantity@9 as ws_quantity, ws_ext_sales_price@10 as ws_ext_sales_price, ws_net_paid@11 as ws_net_paid, w_warehouse_name@0 as w_warehouse_name, w_warehouse_sq_ft@1 as w_warehouse_sq_ft, w_city@2 as w_city, w_county@3 as w_county, w_state@4 as w_state, w_country@5 as w_country] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(warehouse.w_warehouse_sk AS Float64)@7, ws_warehouse_sk@3)], projection=[w_warehouse_name@1, w_warehouse_sq_ft@2, w_city@3, w_county@4, w_state@5, w_country@6, ws_sold_date_sk@8, ws_sold_time_sk@9, ws_ship_mode_sk@10, ws_quantity@12, ws_ext_sales_price@13, ws_net_paid@14] - │ CoalescePartitionsExec - │ [Stage 2] => 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_sold_time_sk, ws_ship_mode_sk, ws_warehouse_sk, ws_quantity, ws_ext_sales_price, ws_net_paid], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-3.parquet]]}, projection=[sm_ship_mode_sk, sm_carrier], file_type=parquet, predicate=sm_carrier@4 = DHL OR sm_carrier@4 = BARIAN, pruning_predicate=sm_carrier_null_count@2 != row_count@3 AND sm_carrier_min@0 <= DHL AND DHL <= sm_carrier_max@1 OR sm_carrier_null_count@2 != row_count@3 AND sm_carrier_min@0 <= BARIAN AND BARIAN <= sm_carrier_max@1, required_guarantees=[sm_carrier in (BARIAN, DHL)] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@3)], projection=[ws_sold_time_sk@1, ws_ship_mode_sk@2, ws_quantity@3, ws_ext_sales_price@4, ws_net_paid@5, w_warehouse_name@6, w_warehouse_sq_ft@7, w_city@8, w_county@9, w_state@10, w_country@11, d_year@13, d_moy@14] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001 + │ 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, d_moy], 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 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ws_sold_date_sk@6 as ws_sold_date_sk, ws_sold_time_sk@7 as ws_sold_time_sk, ws_ship_mode_sk@8 as ws_ship_mode_sk, ws_quantity@9 as ws_quantity, ws_ext_sales_price@10 as ws_ext_sales_price, ws_net_paid@11 as ws_net_paid, w_warehouse_name@0 as w_warehouse_name, w_warehouse_sq_ft@1 as w_warehouse_sq_ft, w_city@2 as w_city, w_county@3 as w_county, w_state@4 as w_state, w_country@5 as w_country] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(warehouse.w_warehouse_sk AS Float64)@7, ws_warehouse_sk@3)], projection=[w_warehouse_name@1, w_warehouse_sq_ft@2, w_city@3, w_county@4, w_state@5, w_country@6, ws_sold_date_sk@8, ws_sold_time_sk@9, ws_ship_mode_sk@10, ws_quantity@12, ws_ext_sales_price@13, ws_net_paid@14] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_sold_time_sk, ws_ship_mode_sk, ws_warehouse_sk, ws_quantity, ws_ext_sales_price, ws_net_paid], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_warehouse_name, w_warehouse_sq_ft, w_city, w_county, w_state, w_country, CAST(w_warehouse_sk@0 AS Float64) as CAST(warehouse.w_warehouse_sk AS Float64)], file_type=parquet + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_warehouse_name, w_warehouse_sq_ft, w_city, w_county, w_state, w_country, CAST(w_warehouse_sk@0 AS Float64) as CAST(warehouse.w_warehouse_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 10 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([w_warehouse_name@0, w_warehouse_sq_ft@1, w_city@2, w_county@3, w_state@4, w_country@5, d_year@6], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[w_warehouse_name@15 as w_warehouse_name, w_warehouse_sq_ft@16 as w_warehouse_sq_ft, w_city@17 as w_city, w_county@18 as w_county, w_state@19 as w_state, w_country@20 as w_country, d_year@21 as d_year], aggr=[sum(CASE WHEN date_dim.d_moy = Int64(1) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(2) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(3) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(4) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(5) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(6) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(7) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(8) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(9) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(10) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(11) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(12) THEN catalog_sales.cs_sales_price * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(1) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(2) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(3) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(4) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(5) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(6) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(7) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(8) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(9) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(10) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(11) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END), sum(CASE WHEN date_dim.d_moy = Int64(12) THEN catalog_sales.cs_net_paid_inc_tax * catalog_sales.cs_quantity ELSE Int64(0) END)], ordering_mode=PartiallySorted([6]) │ ProjectionExec: expr=[d_moy@10 = 1 as __common_expr_14, d_moy@10 = 2 as __common_expr_15, d_moy@10 = 3 as __common_expr_16, d_moy@10 = 4 as __common_expr_17, d_moy@10 = 5 as __common_expr_18, d_moy@10 = 6 as __common_expr_19, d_moy@10 = 7 as __common_expr_20, d_moy@10 = 8 as __common_expr_21, d_moy@10 = 9 as __common_expr_22, d_moy@10 = 10 as __common_expr_23, d_moy@10 = 11 as __common_expr_24, d_moy@10 = 12 as __common_expr_25, cs_quantity@0 as cs_quantity, cs_sales_price@1 as cs_sales_price, cs_net_paid_inc_tax@2 as cs_net_paid_inc_tax, w_warehouse_name@3 as w_warehouse_name, w_warehouse_sq_ft@4 as w_warehouse_sq_ft, w_city@5 as w_city, w_county@6 as w_county, w_state@7 as w_state, w_country@8 as w_country, d_year@9 as d_year] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(ship_mode.sm_ship_mode_sk AS Float64)@1, cs_ship_mode_sk@0)], projection=[cs_quantity@3, cs_sales_price@4, cs_net_paid_inc_tax@5, w_warehouse_name@6, w_warehouse_sq_ft@7, w_city@8, w_county@9, w_state@10, w_country@11, d_year@12, d_moy@13] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_sold_time_sk@0, CAST(time_dim.t_time_sk AS Float64)@1)], projection=[cs_ship_mode_sk@1, cs_quantity@2, cs_sales_price@3, cs_net_paid_inc_tax@4, w_warehouse_name@5, w_warehouse_sq_ft@6, w_city@7, w_county@8, w_state@9, w_country@10, d_year@11, d_moy@12] - │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] │ FilterExec: t_time@1 >= 30838 AND t_time@1 <= 59638, projection=[t_time_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/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_time], file_type=parquet, predicate=t_time@2 >= 30838 AND t_time@2 <= 59638, pruning_predicate=t_time_null_count@1 != row_count@2 AND t_time_max@0 >= 30838 AND t_time_null_count@1 != row_count@2 AND t_time_min@3 <= 59638, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[sm_ship_mode_sk@0 as sm_ship_mode_sk, CAST(sm_ship_mode_sk@0 AS Float64) as CAST(ship_mode.sm_ship_mode_sk AS Float64)] - │ FilterExec: sm_carrier@1 = DHL OR sm_carrier@1 = BARIAN, projection=[sm_ship_mode_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-3.parquet]]}, projection=[sm_ship_mode_sk, sm_carrier], file_type=parquet, predicate=sm_carrier@4 = DHL OR sm_carrier@4 = BARIAN, pruning_predicate=sm_carrier_null_count@2 != row_count@3 AND sm_carrier_min@0 <= DHL AND DHL <= sm_carrier_max@1 OR sm_carrier_null_count@2 != row_count@3 AND sm_carrier_min@0 <= BARIAN AND BARIAN <= sm_carrier_max@1, required_guarantees=[sm_carrier in (BARIAN, DHL)] - └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@3)], projection=[cs_sold_time_sk@1, cs_ship_mode_sk@2, cs_quantity@3, cs_sales_price@4, cs_net_paid_inc_tax@5, w_warehouse_name@6, w_warehouse_sq_ft@7, w_city@8, w_county@9, w_state@10, w_country@11, d_year@13, d_moy@14] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001 + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[sm_ship_mode_sk@0 as sm_ship_mode_sk, CAST(sm_ship_mode_sk@0 AS Float64) as CAST(ship_mode.sm_ship_mode_sk AS Float64)] + │ FilterExec: sm_carrier@1 = DHL OR sm_carrier@1 = BARIAN, projection=[sm_ship_mode_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, d_moy], 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 8 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ ProjectionExec: expr=[cs_sold_date_sk@6 as cs_sold_date_sk, cs_sold_time_sk@7 as cs_sold_time_sk, cs_ship_mode_sk@8 as cs_ship_mode_sk, cs_quantity@9 as cs_quantity, cs_sales_price@10 as cs_sales_price, cs_net_paid_inc_tax@11 as cs_net_paid_inc_tax, w_warehouse_name@0 as w_warehouse_name, w_warehouse_sq_ft@1 as w_warehouse_sq_ft, w_city@2 as w_city, w_county@3 as w_county, w_state@4 as w_state, w_country@5 as w_country] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(warehouse.w_warehouse_sk AS Float64)@7, cs_warehouse_sk@3)], projection=[w_warehouse_name@1, w_warehouse_sq_ft@2, w_city@3, w_county@4, w_state@5, w_country@6, cs_sold_date_sk@8, cs_sold_time_sk@9, cs_ship_mode_sk@10, cs_quantity@12, cs_sales_price@13, cs_net_paid_inc_tax@14] - │ CoalescePartitionsExec - │ [Stage 7] => 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_sold_time_sk, cs_ship_mode_sk, cs_warehouse_sk, cs_quantity, cs_sales_price, cs_net_paid_inc_tax], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-3.parquet]]}, projection=[sm_ship_mode_sk, sm_carrier], file_type=parquet, predicate=sm_carrier@4 = DHL OR sm_carrier@4 = BARIAN, pruning_predicate=sm_carrier_null_count@2 != row_count@3 AND sm_carrier_min@0 <= DHL AND DHL <= sm_carrier_max@1 OR sm_carrier_null_count@2 != row_count@3 AND sm_carrier_min@0 <= BARIAN AND BARIAN <= sm_carrier_max@1, required_guarantees=[sm_carrier in (BARIAN, DHL)] + └────────────────────────────────────────────────── + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@3)], projection=[cs_sold_time_sk@1, cs_ship_mode_sk@2, cs_quantity@3, cs_sales_price@4, cs_net_paid_inc_tax@5, w_warehouse_name@6, w_warehouse_sq_ft@7, w_city@8, w_county@9, w_state@10, w_country@11, d_year@13, d_moy@14] + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001 + │ 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, d_moy], 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 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[cs_sold_date_sk@6 as cs_sold_date_sk, cs_sold_time_sk@7 as cs_sold_time_sk, cs_ship_mode_sk@8 as cs_ship_mode_sk, cs_quantity@9 as cs_quantity, cs_sales_price@10 as cs_sales_price, cs_net_paid_inc_tax@11 as cs_net_paid_inc_tax, w_warehouse_name@0 as w_warehouse_name, w_warehouse_sq_ft@1 as w_warehouse_sq_ft, w_city@2 as w_city, w_county@3 as w_county, w_state@4 as w_state, w_country@5 as w_country] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(warehouse.w_warehouse_sk AS Float64)@7, cs_warehouse_sk@3)], projection=[w_warehouse_name@1, w_warehouse_sq_ft@2, w_city@3, w_county@4, w_state@5, w_country@6, cs_sold_date_sk@8, cs_sold_time_sk@9, cs_ship_mode_sk@10, cs_quantity@12, cs_sales_price@13, cs_net_paid_inc_tax@14] + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_sold_time_sk, cs_ship_mode_sk, cs_warehouse_sk, cs_quantity, cs_sales_price, cs_net_paid_inc_tax], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_warehouse_name, w_warehouse_sq_ft, w_city, w_county, w_state, w_country, CAST(w_warehouse_sk@0 AS Float64) as CAST(warehouse.w_warehouse_sk AS Float64)], file_type=parquet + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_warehouse_name, w_warehouse_sq_ft, w_city, w_county, w_state, w_country, CAST(w_warehouse_sk@0 AS Float64) as CAST(warehouse.w_warehouse_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── "); Ok(()) @@ -7279,37 +7296,37 @@ mod tests { │ AggregateExec: mode=Partial, gby=[(NULL as i_category, NULL as i_class, NULL as i_brand, NULL as i_product_name, NULL as d_year, NULL as d_qoy, NULL as d_moy, NULL as s_store_id), (i_category@7 as i_category, NULL as i_class, NULL as i_brand, NULL as i_product_name, NULL as d_year, NULL as d_qoy, NULL as d_moy, NULL as s_store_id), (i_category@7 as i_category, i_class@6 as i_class, NULL as i_brand, NULL as i_product_name, NULL as d_year, NULL as d_qoy, NULL as d_moy, NULL as s_store_id), (i_category@7 as i_category, i_class@6 as i_class, i_brand@5 as i_brand, NULL as i_product_name, NULL as d_year, NULL as d_qoy, NULL as d_moy, NULL as s_store_id), (i_category@7 as i_category, i_class@6 as i_class, i_brand@5 as i_brand, i_product_name@8 as i_product_name, NULL as d_year, NULL as d_qoy, NULL as d_moy, NULL as s_store_id), (i_category@7 as i_category, i_class@6 as i_class, i_brand@5 as i_brand, i_product_name@8 as i_product_name, d_year@1 as d_year, NULL as d_qoy, NULL as d_moy, NULL as s_store_id), (i_category@7 as i_category, i_class@6 as i_class, i_brand@5 as i_brand, i_product_name@8 as i_product_name, d_year@1 as d_year, d_qoy@3 as d_qoy, NULL as d_moy, NULL as s_store_id), (i_category@7 as i_category, i_class@6 as i_class, i_brand@5 as i_brand, i_product_name@8 as i_product_name, d_year@1 as d_year, d_qoy@3 as d_qoy, d_moy@2 as d_moy, NULL as s_store_id), (i_category@7 as i_category, i_class@6 as i_class, i_brand@5 as i_brand, i_product_name@8 as i_product_name, d_year@1 as d_year, d_qoy@3 as d_qoy, d_moy@2 as d_moy, s_store_id@4 as s_store_id)], aggr=[sum(coalesce(store_sales.ss_sales_price * store_sales.ss_quantity,Int64(0)))] │ ProjectionExec: expr=[CAST(ss_sales_price@1 AS Float64) * ss_quantity@0 as __common_expr_1, d_year@2 as d_year, d_moy@3 as d_moy, d_qoy@4 as d_qoy, s_store_id@5 as s_store_id, i_brand@6 as i_brand, i_class@7 as i_class, i_category@8 as i_category, i_product_name@9 as i_product_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_item_sk@0, i_item_sk@0)], projection=[ss_quantity@1, ss_sales_price@2, d_year@3, d_moy@4, d_qoy@5, s_store_id@6, i_brand@8, i_class@9, i_category@10, i_product_name@11] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_brand, i_class, i_category, i_product_name], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_quantity@2 as ss_quantity, ss_sales_price@3 as ss_sales_price, d_year@4 as d_year, d_moy@5 as d_moy, d_qoy@6 as d_qoy, s_store_id@0 as s_store_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, ss_store_sk@1)], projection=[s_store_id@1, ss_item_sk@3, ss_quantity@5, ss_sales_price@6, d_year@7, d_moy@8, d_qoy@9] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[ss_item_sk@3 as ss_item_sk, ss_store_sk@4 as ss_store_sk, ss_quantity@5 as ss_quantity, ss_sales_price@6 as ss_sales_price, d_year@0 as d_year, d_moy@1 as d_moy, d_qoy@2 as d_qoy] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@4, ss_sold_date_sk@0)], projection=[d_year@1, d_moy@2, d_qoy@3, ss_item_sk@6, ss_store_sk@7, ss_quantity@8, ss_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/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_quantity, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, d_qoy@3 as d_qoy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_month_seq@1 >= 1200 AND d_month_seq@1 <= 1211, projection=[d_date_sk@0, d_year@2, d_moy@3, d_qoy@4] - │ 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_month_seq, d_year, d_moy, d_qoy], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_quantity@2 as ss_quantity, ss_sales_price@3 as ss_sales_price, d_year@4 as d_year, d_moy@5 as d_moy, d_qoy@6 as d_qoy, s_store_id@0 as s_store_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, ss_store_sk@1)], projection=[s_store_id@1, ss_item_sk@3, ss_quantity@5, ss_sales_price@6, d_year@7, d_moy@8, d_qoy@9] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ ProjectionExec: expr=[ss_item_sk@3 as ss_item_sk, ss_store_sk@4 as ss_store_sk, ss_quantity@5 as ss_quantity, ss_sales_price@6 as ss_sales_price, d_year@0 as d_year, d_moy@1 as d_moy, d_qoy@2 as d_qoy] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@4, ss_sold_date_sk@0)], projection=[d_year@1, d_moy@2, d_qoy@3, ss_item_sk@6, ss_store_sk@7, ss_quantity@8, ss_sales_price@9] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_store_sk, ss_quantity, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, d_moy@2 as d_moy, d_qoy@3 as d_qoy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_month_seq@1 >= 1200 AND d_month_seq@1 <= 1211, projection=[d_date_sk@0, d_year@2, d_moy@3, d_qoy@4] + │ 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_month_seq, d_year, d_moy, d_qoy], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── "#); Ok(()) @@ -7326,76 +7343,76 @@ mod tests { │ 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 + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@4)], projection=[ss_ticket_number@0, bought_city@2, extended_price@3, list_price@4, extended_tax@5, c_current_addr_sk@7, c_first_name@8, c_last_name@9] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk, c_first_name, c_last_name], file_type=parquet + ┌───── Stage 7 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@1, CAST(customer.c_customer_sk AS Float64)@4)], projection=[ss_ticket_number@0, bought_city@2, extended_price@3, list_price@4, extended_tax@5, c_current_addr_sk@7, c_first_name@8, c_last_name@9] + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, input_tasks=2 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_addr_sk@1 as c_current_addr_sk, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, CAST(c_customer_sk@0 AS Float64) as CAST(customer.c_customer_sk AS Float64)] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_addr_sk, c_first_name, c_last_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ ProjectionExec: expr=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk, ca_city@3 as bought_city, sum(store_sales.ss_ext_sales_price)@4 as extended_price, sum(store_sales.ss_ext_list_price)@5 as list_price, sum(store_sales.ss_ext_tax)@6 as extended_tax] - │ AggregateExec: mode=FinalPartitioned, gby=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk, ss_addr_sk@2 as ss_addr_sk, ca_city@3 as ca_city], aggr=[sum(store_sales.ss_ext_sales_price), sum(store_sales.ss_ext_list_price), sum(store_sales.ss_ext_tax)] - │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 6 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk, ca_city@3 as bought_city, sum(store_sales.ss_ext_sales_price)@4 as extended_price, sum(store_sales.ss_ext_list_price)@5 as list_price, sum(store_sales.ss_ext_tax)@6 as extended_tax] + │ AggregateExec: mode=FinalPartitioned, gby=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk, ss_addr_sk@2 as ss_addr_sk, ca_city@3 as ca_city], aggr=[sum(store_sales.ss_ext_sales_price), sum(store_sales.ss_ext_list_price), sum(store_sales.ss_ext_tax)] + │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([ss_ticket_number@0, ss_customer_sk@1, ss_addr_sk@2, ca_city@3], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[ss_ticket_number@2 as ss_ticket_number, ss_customer_sk@0 as ss_customer_sk, ss_addr_sk@1 as ss_addr_sk, ca_city@6 as ca_city], aggr=[sum(store_sales.ss_ext_sales_price), sum(store_sales.ss_ext_list_price), sum(store_sales.ss_ext_tax)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@2)], projection=[ss_customer_sk@0, ss_addr_sk@1, ss_ticket_number@2, ss_ext_sales_price@3, ss_ext_list_price@4, ss_ext_tax@5, ca_city@7] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_city@1 as ca_city, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_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/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 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_customer_sk@2, ss_addr_sk@4, ss_ticket_number@5, ss_ext_sales_price@6, ss_ext_list_price@7, ss_ext_tax@8] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@3)], projection=[ss_customer_sk@2, ss_hdemo_sk@3, ss_addr_sk@4, ss_ticket_number@6, ss_ext_sales_price@7, ss_ext_list_price@8, ss_ext_tax@9] - │ 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, ss_sold_date_sk@0)], projection=[ss_customer_sk@3, ss_hdemo_sk@4, ss_addr_sk@5, ss_store_sk@6, ss_ticket_number@7, ss_ext_sales_price@8, ss_ext_list_price@9, ss_ext_tax@10] - │ 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_date_sk, ss_customer_sk, ss_hdemo_sk, ss_addr_sk, ss_store_sk, ss_ticket_number, ss_ext_sales_price, ss_ext_list_price, ss_ext_tax], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_customer_sk@2, ss_addr_sk@4, ss_ticket_number@5, ss_ext_sales_price@6, ss_ext_list_price@7, ss_ext_tax@8] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@3)], projection=[ss_customer_sk@2, ss_hdemo_sk@3, ss_addr_sk@4, ss_ticket_number@6, ss_ext_sales_price@7, ss_ext_list_price@8, ss_ext_tax@9] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_customer_sk@3, ss_hdemo_sk@4, ss_addr_sk@5, ss_store_sk@6, ss_ticket_number@7, ss_ext_sales_price@8, ss_ext_list_price@9, ss_ext_tax@10] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_customer_sk, ss_hdemo_sk, ss_addr_sk, ss_store_sk, ss_ticket_number, ss_ext_sales_price, ss_ext_list_price, ss_ext_tax], 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 - │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] - │ FilterExec: hd_dep_count@1 = 4 OR hd_vehicle_count@2 = 3, projection=[hd_demo_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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 OR hd_vehicle_count@4 = 3, 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 OR hd_vehicle_count_null_count@6 != row_count@3 AND hd_vehicle_count_min@4 <= 3 AND 3 <= hd_vehicle_count_max@5, required_guarantees=[] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] + │ FilterExec: hd_dep_count@1 = 4 OR hd_vehicle_count@2 = 3, projection=[hd_demo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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 OR hd_vehicle_count@4 = 3, 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 OR hd_vehicle_count_null_count@6 != row_count@3 AND hd_vehicle_count_min@4 <= 3 AND 3 <= hd_vehicle_count_max@5, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_city@1 = Fairview OR s_city@1 = Midway, projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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_city], file_type=parquet, predicate=s_city@22 = Fairview OR s_city@22 = Midway, pruning_predicate=s_city_null_count@2 != row_count@3 AND s_city_min@0 <= Fairview AND Fairview <= s_city_max@1 OR s_city_null_count@2 != row_count@3 AND s_city_min@0 <= Midway AND Midway <= s_city_max@1, required_guarantees=[s_city in (Fairview, Midway)] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_city@1 = Fairview OR s_city@1 = Midway, projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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_city], file_type=parquet, predicate=s_city@22 = Fairview OR s_city@22 = Midway, pruning_predicate=s_city_null_count@2 != row_count@3 AND s_city_min@0 <= Fairview AND Fairview <= s_city_max@1 OR s_city_null_count@2 != row_count@3 AND s_city_min@0 <= Midway AND Midway <= s_city_max@1, required_guarantees=[s_city in (Fairview, Midway)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_dom@2 >= 1 AND d_dom@2 <= 2 AND (d_year@1 = 1999 OR d_year@1 = 2000 OR d_year@1 = 2001), projection=[d_date_sk@0] - │ 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_year, d_dom], file_type=parquet, predicate=d_dom@9 >= 1 AND d_dom@9 <= 2 AND (d_year@6 = 1999 OR d_year@6 = 2000 OR d_year@6 = 2001), pruning_predicate=d_dom_null_count@1 != row_count@2 AND d_dom_max@0 >= 1 AND d_dom_null_count@1 != row_count@2 AND d_dom_min@3 <= 2 AND (d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5 OR d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5 OR d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5), required_guarantees=[d_year in (1999, 2000, 2001)] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_dom@2 >= 1 AND d_dom@2 <= 2 AND (d_year@1 = 1999 OR d_year@1 = 2000 OR d_year@1 = 2001), projection=[d_date_sk@0] + │ 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_year, d_dom], file_type=parquet, predicate=d_dom@9 >= 1 AND d_dom@9 <= 2 AND (d_year@6 = 1999 OR d_year@6 = 2000 OR d_year@6 = 2001), pruning_predicate=d_dom_null_count@1 != row_count@2 AND d_dom_max@0 >= 1 AND d_dom_null_count@1 != row_count@2 AND d_dom_min@3 <= 2 AND (d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5 OR d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5 OR d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5), required_guarantees=[d_year in (1999, 2000, 2001)] └────────────────────────────────────────────────── "); Ok(()) @@ -7418,86 +7435,86 @@ mod tests { │ 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 │ AggregateExec: mode=Partial, 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))] │ HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(cs_ship_customer_sk@0, CAST(c.c_customer_sk AS Float64)@6)], projection=[cd_gender@1, cd_marital_status@2, cd_education_status@3, cd_purchase_estimate@4, cd_credit_rating@5] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, cd_gender@1 as cd_gender, cd_marital_status@2 as cd_marital_status, cd_education_status@3 as cd_education_status, cd_purchase_estimate@4 as cd_purchase_estimate, cd_credit_rating@5 as cd_credit_rating, CAST(c_customer_sk@0 AS Float64) as CAST(c.c_customer_sk AS Float64)] │ HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(ws_bill_customer_sk@0, CAST(c.c_customer_sk AS Float64)@6)], projection=[c_customer_sk@0, cd_gender@1, cd_marital_status@2, cd_education_status@3, cd_purchase_estimate@4, cd_credit_rating@5] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, cd_gender@1 as cd_gender, cd_marital_status@2 as cd_marital_status, cd_education_status@3 as cd_education_status, cd_purchase_estimate@4 as cd_purchase_estimate, cd_credit_rating@5 as cd_credit_rating, CAST(c_customer_sk@0 AS Float64) as CAST(c.c_customer_sk AS Float64)] │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(ss_customer_sk@0, CAST(c.c_customer_sk AS Float64)@6)], projection=[c_customer_sk@0, cd_gender@1, cd_marital_status@2, cd_education_status@3, cd_purchase_estimate@4, cd_credit_rating@5] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, cd_gender@1 as cd_gender, cd_marital_status@2 as cd_marital_status, cd_education_status@3 as cd_education_status, cd_purchase_estimate@4 as cd_purchase_estimate, cd_credit_rating@5 as cd_credit_rating, CAST(c_customer_sk@0 AS Float64) as CAST(c.c_customer_sk AS Float64)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_cdemo_sk@1, CAST(customer_demographics.cd_demo_sk AS Float64)@6)], projection=[c_customer_sk@0, cd_gender@3, cd_marital_status@4, cd_education_status@5, cd_purchase_estimate@6, cd_credit_rating@7] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_gender, cd_marital_status, cd_education_status, cd_purchase_estimate, cd_credit_rating, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_ship_customer_sk@3] - │ CoalescePartitionsExec - │ [Stage 1] => 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_ship_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_ship_customer_sk@3] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_ship_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 AND d_moy@2 >= 4 AND d_moy@2 <= 6, projection=[d_date_sk@0] - │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 >= 4 AND d_moy@8 <= 6, 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@5 != row_count@3 AND d_moy_max@4 >= 4 AND d_moy_null_count@5 != row_count@3 AND d_moy_min@6 <= 6, required_guarantees=[d_year in (2001)] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 AND d_moy@2 >= 4 AND d_moy@2 <= 6, projection=[d_date_sk@0] + │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 >= 4 AND d_moy@8 <= 6, 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@5 != row_count@3 AND d_moy_max@4 >= 4 AND d_moy_null_count@5 != row_count@3 AND d_moy_min@6 <= 6, required_guarantees=[d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_bill_customer_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_date_sk, ws_bill_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_bill_customer_sk@3] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 AND d_moy@2 >= 4 AND d_moy@2 <= 6, projection=[d_date_sk@0] - │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 >= 4 AND d_moy@8 <= 6, 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@5 != row_count@3 AND d_moy_max@4 >= 4 AND d_moy_null_count@5 != row_count@3 AND d_moy_min@6 <= 6, required_guarantees=[d_year in (2001)] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 AND d_moy@2 >= 4 AND d_moy@2 <= 6, projection=[d_date_sk@0] + │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 >= 4 AND d_moy@8 <= 6, 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@5 != row_count@3 AND d_moy_max@4 >= 4 AND d_moy_null_count@5 != row_count@3 AND d_moy_min@6 <= 6, required_guarantees=[d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_customer_sk@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/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_customer_sk@3] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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 AND d_moy@2 >= 4 AND d_moy@2 <= 6, projection=[d_date_sk@0] - │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 >= 4 AND d_moy@8 <= 6, 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@5 != row_count@3 AND d_moy_max@4 >= 4 AND d_moy_null_count@5 != row_count@3 AND d_moy_min@6 <= 6, required_guarantees=[d_year in (2001)] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 AND d_moy@2 >= 4 AND d_moy@2 <= 6, projection=[d_date_sk@0] + │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 2001 AND d_moy@8 >= 4 AND d_moy@8 <= 6, 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@5 != row_count@3 AND d_moy_max@4 >= 4 AND d_moy_null_count@5 != row_count@3 AND d_moy_min@6 <= 6, required_guarantees=[d_year in (2001)] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@2)], projection=[c_customer_sk@1, c_current_cdemo_sk@2] - │ 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/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_addr_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@2)], projection=[c_customer_sk@1, c_current_cdemo_sk@2] + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_addr_sk], 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 - │ FilterExec: ca_state@1 = KY OR ca_state@1 = GA OR ca_state@1 = NM, projection=[ca_address_sk@0] - │ 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_state], file_type=parquet, predicate=ca_state@8 = KY OR ca_state@8 = GA OR ca_state@8 = NM, pruning_predicate=ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= KY AND KY <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= GA AND GA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= NM AND NM <= ca_state_max@1, required_guarantees=[ca_state in (GA, KY, NM)] + ┌───── Stage 7 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ FilterExec: ca_state@1 = KY OR ca_state@1 = GA OR ca_state@1 = NM, projection=[ca_address_sk@0] + │ 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_state], file_type=parquet, predicate=ca_state@8 = KY OR ca_state@8 = GA OR ca_state@8 = NM, pruning_predicate=ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= KY AND KY <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= GA AND GA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= NM AND NM <= ca_state_max@1, required_guarantees=[ca_state in (GA, KY, NM)] └────────────────────────────────────────────────── "); Ok(()) @@ -7528,60 +7545,60 @@ mod tests { │ AggregateExec: mode=Partial, gby=[i_brand@1 as i_brand, i_brand_id@0 as i_brand_id, t_hour@3 as t_hour, t_minute@4 as t_minute], aggr=[sum(tmp.ext_price)] │ ProjectionExec: expr=[i_brand_id@2 as i_brand_id, i_brand@3 as i_brand, ext_price@4 as ext_price, t_hour@0 as t_hour, t_minute@1 as t_minute] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(time_dim.t_time_sk AS Float64)@3, time_sk@3)], projection=[t_hour@1, t_minute@2, i_brand_id@4, i_brand@5, ext_price@6] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=2 │ ProjectionExec: expr=[i_brand_id@2 as i_brand_id, i_brand@3 as i_brand, ext_price@0 as ext_price, time_sk@1 as time_sk] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(sold_item_sk@1, i_item_sk@0)], projection=[ext_price@0, time_sk@2, i_brand_id@4, i_brand@5] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=12, input_tasks=4 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ FilterExec: i_manager_id@3 = 1, projection=[i_item_sk@0, i_brand_id@1, i_brand@2] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 │ PartitionIsolatorExec: t0:[p0,__,__,__] t1:[__,p0,__,__] t2:[__,__,p0,__] t3:[__,__,__,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_brand_id, i_brand, i_manager_id], file_type=parquet, predicate=i_manager_id@20 = 1 AND DynamicFilter [ empty ], pruning_predicate=i_manager_id_null_count@2 != row_count@3 AND i_manager_id_min@0 <= 1 AND 1 <= i_manager_id_max@1, required_guarantees=[i_manager_id in (1)] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p12..p23] - │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 - │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, t_hour@1 as t_hour, t_minute@2 as t_minute, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] - │ FilterExec: t_meal_time@3 = breakfast OR t_meal_time@3 = dinner, projection=[t_time_sk@0, t_hour@1, t_minute@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/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute, t_meal_time], file_type=parquet, predicate=t_meal_time@9 = breakfast OR t_meal_time@9 = dinner, pruning_predicate=t_meal_time_null_count@2 != row_count@3 AND t_meal_time_min@0 <= breakfast AND breakfast <= t_meal_time_max@1 OR t_meal_time_null_count@2 != row_count@3 AND t_meal_time_min@0 <= dinner AND dinner <= t_meal_time_max@1, required_guarantees=[t_meal_time in (breakfast, dinner)] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p12..p23] t2:[p24..p35] t3:[p36..p47] - │ BroadcastExec: input_partitions=3, consumer_tasks=4, output_partitions=12 - │ DistributedUnionExec: t0:[c0] t1:[c1] t2:[c2(0/2)] t3:[c2(1/2)] - │ ProjectionExec: expr=[ws_ext_sales_price@2 as ext_price, ws_item_sk@1 as sold_item_sk, ws_sold_time_sk@0 as time_sk] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_sold_time_sk@3, ws_item_sk@4, ws_ext_sales_price@5] - │ 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_moy@2 = 11 AND d_year@1 = 1999, 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, d_moy], file_type=parquet, predicate=d_moy@8 = 11 AND d_year@6 = 1999, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5, required_guarantees=[d_moy in (11), d_year in (1999)] - │ 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_sold_date_sk, ws_sold_time_sk, ws_item_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] - │ ProjectionExec: expr=[cs_ext_sales_price@2 as ext_price, cs_item_sk@1 as sold_item_sk, cs_sold_time_sk@0 as time_sk] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_sold_time_sk@3, cs_item_sk@4, cs_ext_sales_price@5] - │ 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_moy@2 = 11 AND d_year@1 = 1999, 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, d_moy], file_type=parquet, predicate=d_moy@8 = 11 AND d_year@6 = 1999, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5, required_guarantees=[d_moy in (11), d_year in (1999)] - │ 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_sold_time_sk, cs_item_sk, cs_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] - │ ProjectionExec: expr=[ss_ext_sales_price@2 as ext_price, ss_item_sk@1 as sold_item_sk, ss_sold_time_sk@0 as time_sk] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_sold_time_sk@3, ss_item_sk@4, ss_ext_sales_price@5] - │ CoalescePartitionsExec - │ [Stage 2] => 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/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_sold_time_sk, ss_item_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ 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_moy@2 = 11 AND d_year@1 = 1999, projection=[d_date_sk@0] - │ 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_year, d_moy], file_type=parquet, predicate=d_moy@8 = 11 AND d_year@6 = 1999, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5, required_guarantees=[d_moy in (11), d_year in (1999)] + ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, t_hour@1 as t_hour, t_minute@2 as t_minute, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] + │ FilterExec: t_meal_time@3 = breakfast OR t_meal_time@3 = dinner, projection=[t_time_sk@0, t_hour@1, t_minute@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/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute, t_meal_time], file_type=parquet, predicate=t_meal_time@9 = breakfast OR t_meal_time@9 = dinner, pruning_predicate=t_meal_time_null_count@2 != row_count@3 AND t_meal_time_min@0 <= breakfast AND breakfast <= t_meal_time_max@1 OR t_meal_time_null_count@2 != row_count@3 AND t_meal_time_min@0 <= dinner AND dinner <= t_meal_time_max@1, required_guarantees=[t_meal_time in (breakfast, dinner)] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ DistributedUnionExec: t0:[c0] t1:[c1] t2:[c2(0/2)] t3:[c2(1/2)] + │ ProjectionExec: expr=[ws_ext_sales_price@2 as ext_price, ws_item_sk@1 as sold_item_sk, ws_sold_time_sk@0 as time_sk] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ws_sold_date_sk@0)], projection=[ws_sold_time_sk@3, ws_item_sk@4, ws_ext_sales_price@5] + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec + │ 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_moy@2 = 11 AND d_year@1 = 1999, 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, d_moy], file_type=parquet, predicate=d_moy@8 = 11 AND d_year@6 = 1999, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5, required_guarantees=[d_moy in (11), d_year in (1999)] + │ 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_sold_date_sk, ws_sold_time_sk, ws_item_sk, ws_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + │ ProjectionExec: expr=[cs_ext_sales_price@2 as ext_price, cs_item_sk@1 as sold_item_sk, cs_sold_time_sk@0 as time_sk] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_sold_time_sk@3, cs_item_sk@4, cs_ext_sales_price@5] + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec + │ 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_moy@2 = 11 AND d_year@1 = 1999, 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, d_moy], file_type=parquet, predicate=d_moy@8 = 11 AND d_year@6 = 1999, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5, required_guarantees=[d_moy in (11), d_year in (1999)] + │ 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_sold_time_sk, cs_item_sk, cs_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + │ ProjectionExec: expr=[ss_ext_sales_price@2 as ext_price, ss_item_sk@1 as sold_item_sk, ss_sold_time_sk@0 as time_sk] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_sold_time_sk@3, ss_item_sk@4, ss_ext_sales_price@5] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, input_tasks=2 + │ PartitionIsolatorExec: t0:[p0,p1,p2,__,__,__] t1:[__,__,__,p0,p1,p2] + │ 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_sold_time_sk, ss_item_sk, ss_ext_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ 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_moy@2 = 11 AND d_year@1 = 1999, projection=[d_date_sk@0] + │ 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_year, d_moy], file_type=parquet, predicate=d_moy@8 = 11 AND d_year@6 = 1999, pruning_predicate=d_moy_null_count@2 != row_count@3 AND d_moy_min@0 <= 11 AND 11 <= d_moy_max@1 AND d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5, required_guarantees=[d_moy in (11), d_year in (1999)] └────────────────────────────────────────────────── "); Ok(()) @@ -7605,58 +7622,58 @@ mod tests { ┌───── 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 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(inv_date_sk@4, d_date_sk@0), (d_week_seq@8, d_week_seq@1)], projection=[cs_ship_date_sk@0, cs_item_sk@1, cs_promo_sk@2, cs_order_number@3, w_warehouse_name@5, i_item_desc@6, d_date@7, d_week_seq@8] + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_week_seq], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(inv_date_sk@4, d_date_sk@0), (d_week_seq@8, d_week_seq@1)], projection=[cs_ship_date_sk@0, cs_item_sk@1, cs_promo_sk@2, cs_order_number@3, w_warehouse_name@5, i_item_desc@6, d_date@7, d_week_seq@8] - │ CoalescePartitionsExec - │ [Stage 10] => 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/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_week_seq], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[cs_ship_date_sk@2 as cs_ship_date_sk, cs_item_sk@3 as cs_item_sk, cs_promo_sk@4 as cs_promo_sk, cs_order_number@5 as cs_order_number, inv_date_sk@6 as inv_date_sk, w_warehouse_name@7 as w_warehouse_name, i_item_desc@8 as i_item_desc, d_date@0 as d_date, d_week_seq@1 as d_week_seq] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@3, cs_sold_date_sk@0)], projection=[d_date@1, d_week_seq@2, cs_ship_date_sk@5, cs_item_sk@6, cs_promo_sk@7, cs_order_number@8, inv_date_sk@9, w_warehouse_name@10, i_item_desc@11] - │ 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, cs_bill_hdemo_sk@2)], projection=[cs_sold_date_sk@2, cs_ship_date_sk@3, cs_item_sk@5, cs_promo_sk@6, cs_order_number@7, inv_date_sk@8, w_warehouse_name@9, i_item_desc@10] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, cs_bill_cdemo_sk@2)], projection=[cs_sold_date_sk@2, cs_ship_date_sk@3, cs_bill_hdemo_sk@5, cs_item_sk@6, cs_promo_sk@7, cs_order_number@8, inv_date_sk@9, w_warehouse_name@10, i_item_desc@11] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, d_week_seq@2 as d_week_seq, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] - │ FilterExec: d_year@3 = 1999, projection=[d_date_sk@0, d_date@1, d_week_seq@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/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, d_week_seq, d_year], file_type=parquet, predicate=d_year@6 = 1999, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1, required_guarantees=[d_year in (1999)] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] - │ FilterExec: hd_buy_potential@1 = >10000, projection=[hd_demo_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_buy_potential], file_type=parquet, predicate=hd_buy_potential@2 = >10000, pruning_predicate=hd_buy_potential_null_count@2 != row_count@3 AND hd_buy_potential_min@0 <= >10000 AND >10000 <= hd_buy_potential_max@1, required_guarantees=[hd_buy_potential in (>10000)] + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[cs_ship_date_sk@2 as cs_ship_date_sk, cs_item_sk@3 as cs_item_sk, cs_promo_sk@4 as cs_promo_sk, cs_order_number@5 as cs_order_number, inv_date_sk@6 as inv_date_sk, w_warehouse_name@7 as w_warehouse_name, i_item_desc@8 as i_item_desc, d_date@0 as d_date, d_week_seq@1 as d_week_seq] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(d1.d_date_sk AS Float64)@3, cs_sold_date_sk@0)], projection=[d_date@1, d_week_seq@2, cs_ship_date_sk@5, cs_item_sk@6, cs_promo_sk@7, cs_order_number@8, inv_date_sk@9, w_warehouse_name@10, i_item_desc@11] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, cs_bill_hdemo_sk@2)], projection=[cs_sold_date_sk@2, cs_ship_date_sk@3, cs_item_sk@5, cs_promo_sk@6, cs_order_number@7, inv_date_sk@8, w_warehouse_name@9, i_item_desc@10] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(customer_demographics.cd_demo_sk AS Float64)@1, cs_bill_cdemo_sk@2)], projection=[cs_sold_date_sk@2, cs_ship_date_sk@3, cs_bill_hdemo_sk@5, cs_item_sk@6, cs_promo_sk@7, cs_order_number@8, inv_date_sk@9, w_warehouse_name@10, i_item_desc@11] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_date@1 as d_date, d_week_seq@2 as d_week_seq, CAST(d_date_sk@0 AS Float64) as CAST(d1.d_date_sk AS Float64)] + │ FilterExec: d_year@3 = 1999, projection=[d_date_sk@0, d_date@1, d_week_seq@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/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, d_week_seq, d_year], file_type=parquet, predicate=d_year@6 = 1999, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1, required_guarantees=[d_year in (1999)] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] + │ FilterExec: hd_buy_potential@1 = >10000, projection=[hd_demo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_buy_potential], file_type=parquet, predicate=hd_buy_potential@2 = >10000, pruning_predicate=hd_buy_potential_null_count@2 != row_count@3 AND hd_buy_potential_min@0 <= >10000 AND >10000 <= hd_buy_potential_max@1, required_guarantees=[hd_buy_potential in (>10000)] └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ RepartitionExec: partitioning=Hash([CAST(customer_demographics.cd_demo_sk AS Float64)@1], 9), input_partitions=2 @@ -7669,25 +7686,25 @@ mod tests { │ RepartitionExec: partitioning=Hash([cs_bill_cdemo_sk@2], 9), input_partitions=3 │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_ship_date_sk@2 as cs_ship_date_sk, cs_bill_cdemo_sk@3 as cs_bill_cdemo_sk, cs_bill_hdemo_sk@4 as cs_bill_hdemo_sk, cs_item_sk@5 as cs_item_sk, cs_promo_sk@6 as cs_promo_sk, cs_order_number@7 as cs_order_number, inv_date_sk@8 as inv_date_sk, w_warehouse_name@9 as w_warehouse_name, i_item_desc@0 as i_item_desc] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@4)], projection=[i_item_desc@1, cs_sold_date_sk@2, cs_ship_date_sk@3, cs_bill_cdemo_sk@4, cs_bill_hdemo_sk@5, cs_item_sk@6, cs_promo_sk@7, cs_order_number@8, inv_date_sk@9, w_warehouse_name@10] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_ship_date_sk@2 as cs_ship_date_sk, cs_bill_cdemo_sk@3 as cs_bill_cdemo_sk, cs_bill_hdemo_sk@4 as cs_bill_hdemo_sk, cs_item_sk@5 as cs_item_sk, cs_promo_sk@6 as cs_promo_sk, cs_order_number@7 as cs_order_number, inv_date_sk@8 as inv_date_sk, w_warehouse_name@0 as w_warehouse_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(w_warehouse_sk@0, inv_warehouse_sk@8)], projection=[w_warehouse_name@1, cs_sold_date_sk@2, cs_ship_date_sk@3, cs_bill_cdemo_sk@4, cs_bill_hdemo_sk@5, cs_item_sk@6, cs_promo_sk@7, cs_order_number@8, inv_date_sk@9] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(cs_item_sk@4, inv_item_sk@1)], filter=inv_quantity_on_hand@1 < cs_quantity@0, projection=[cs_sold_date_sk@0, cs_ship_date_sk@1, cs_bill_cdemo_sk@2, cs_bill_hdemo_sk@3, cs_item_sk@4, cs_promo_sk@5, cs_order_number@6, inv_date_sk@8, inv_warehouse_sk@10] │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ [Stage 8] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_desc], file_type=parquet + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_desc], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_warehouse_name], file_type=parquet + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[w_warehouse_sk, w_warehouse_name], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ RepartitionExec: partitioning=Hash([cs_item_sk@4], 9), input_partitions=2 @@ -7714,58 +7731,58 @@ mod tests { │ 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 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 - │ ProjectionExec: expr=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk, count(Int64(1))@2 as cnt] - │ FilterExec: count(Int64(1))@2 >= 1 AND count(Int64(1))@2 <= 5 - │ AggregateExec: mode=FinalPartitioned, gby=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk], aggr=[count(Int64(1))] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 5 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk, count(Int64(1))@2 as cnt] + │ FilterExec: count(Int64(1))@2 >= 1 AND count(Int64(1))@2 <= 5 + │ AggregateExec: mode=FinalPartitioned, gby=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk], aggr=[count(Int64(1))] + │ [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_ticket_number@0, ss_customer_sk@1], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[ss_ticket_number@1 as ss_ticket_number, ss_customer_sk@0 as ss_customer_sk], aggr=[count(Int64(1))] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_customer_sk@2, ss_ticket_number@4] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@2)], projection=[ss_customer_sk@2, ss_hdemo_sk@3, ss_ticket_number@5] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_customer_sk@3, ss_hdemo_sk@4, ss_store_sk@5, ss_ticket_number@6] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_customer_sk, ss_hdemo_sk, ss_store_sk, ss_ticket_number], 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 - │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] - │ FilterExec: (hd_buy_potential@1 = Unknown OR hd_buy_potential@1 = >10000) AND hd_vehicle_count@3 > 0 AND CASE WHEN hd_vehicle_count@3 > 0 THEN CAST(hd_dep_count@2 AS Float64) / CAST(hd_vehicle_count@3 AS Float64) END > 1, projection=[hd_demo_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_buy_potential, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=(hd_buy_potential@2 = Unknown OR hd_buy_potential@2 = >10000) AND hd_vehicle_count@4 > 0 AND CASE WHEN hd_vehicle_count@4 > 0 THEN CAST(hd_dep_count@3 AS Float64) / CAST(hd_vehicle_count@4 AS Float64) END > 1, pruning_predicate=(hd_buy_potential_null_count@2 != row_count@3 AND hd_buy_potential_min@0 <= Unknown AND Unknown <= hd_buy_potential_max@1 OR hd_buy_potential_null_count@2 != row_count@3 AND hd_buy_potential_min@0 <= >10000 AND >10000 <= hd_buy_potential_max@1) AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_max@4 > 0, required_guarantees=[hd_buy_potential in (>10000, Unknown)] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_county@1 IN (SET) ([Orange County, Bronx County, Franklin Parish, Williamson County]), projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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_county], file_type=parquet, predicate=s_county@23 IN (SET) ([Orange County, Bronx County, Franklin Parish, Williamson County]), pruning_predicate=s_county_null_count@2 != row_count@3 AND s_county_min@0 <= Orange County AND Orange County <= s_county_max@1 OR s_county_null_count@2 != row_count@3 AND s_county_min@0 <= Bronx County AND Bronx County <= s_county_max@1 OR s_county_null_count@2 != row_count@3 AND s_county_min@0 <= Franklin Parish AND Franklin Parish <= s_county_max@1 OR s_county_null_count@2 != row_count@3 AND s_county_min@0 <= Williamson County AND Williamson County <= s_county_max@1, required_guarantees=[s_county in (Bronx County, Franklin Parish, Orange County, Williamson County)] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] + │ FilterExec: (hd_buy_potential@1 = Unknown OR hd_buy_potential@1 = >10000) AND hd_vehicle_count@3 > 0 AND CASE WHEN hd_vehicle_count@3 > 0 THEN CAST(hd_dep_count@2 AS Float64) / CAST(hd_vehicle_count@3 AS Float64) END > 1, projection=[hd_demo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_buy_potential, hd_dep_count, hd_vehicle_count], file_type=parquet, predicate=(hd_buy_potential@2 = Unknown OR hd_buy_potential@2 = >10000) AND hd_vehicle_count@4 > 0 AND CASE WHEN hd_vehicle_count@4 > 0 THEN CAST(hd_dep_count@3 AS Float64) / CAST(hd_vehicle_count@4 AS Float64) END > 1, pruning_predicate=(hd_buy_potential_null_count@2 != row_count@3 AND hd_buy_potential_min@0 <= Unknown AND Unknown <= hd_buy_potential_max@1 OR hd_buy_potential_null_count@2 != row_count@3 AND hd_buy_potential_min@0 <= >10000 AND >10000 <= hd_buy_potential_max@1) AND hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_max@4 > 0, required_guarantees=[hd_buy_potential in (>10000, Unknown)] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_county@1 IN (SET) ([Orange County, Bronx County, Franklin Parish, Williamson County]), projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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_county], file_type=parquet, predicate=s_county@23 IN (SET) ([Orange County, Bronx County, Franklin Parish, Williamson County]), pruning_predicate=s_county_null_count@2 != row_count@3 AND s_county_min@0 <= Orange County AND Orange County <= s_county_max@1 OR s_county_null_count@2 != row_count@3 AND s_county_min@0 <= Bronx County AND Bronx County <= s_county_max@1 OR s_county_null_count@2 != row_count@3 AND s_county_min@0 <= Franklin Parish AND Franklin Parish <= s_county_max@1 OR s_county_null_count@2 != row_count@3 AND s_county_min@0 <= Williamson County AND Williamson County <= s_county_max@1, required_guarantees=[s_county in (Bronx County, Franklin Parish, Orange County, Williamson County)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_dom@2 >= 1 AND d_dom@2 <= 2 AND (d_year@1 = 1999 OR d_year@1 = 2000 OR d_year@1 = 2001), projection=[d_date_sk@0] - │ 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_year, d_dom], file_type=parquet, predicate=d_dom@9 >= 1 AND d_dom@9 <= 2 AND (d_year@6 = 1999 OR d_year@6 = 2000 OR d_year@6 = 2001), pruning_predicate=d_dom_null_count@1 != row_count@2 AND d_dom_max@0 >= 1 AND d_dom_null_count@1 != row_count@2 AND d_dom_min@3 <= 2 AND (d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5 OR d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5 OR d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5), required_guarantees=[d_year in (1999, 2000, 2001)] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_dom@2 >= 1 AND d_dom@2 <= 2 AND (d_year@1 = 1999 OR d_year@1 = 2000 OR d_year@1 = 2001), projection=[d_date_sk@0] + │ 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_year, d_dom], file_type=parquet, predicate=d_dom@9 >= 1 AND d_dom@9 <= 2 AND (d_year@6 = 1999 OR d_year@6 = 2000 OR d_year@6 = 2001), pruning_predicate=d_dom_null_count@1 != row_count@2 AND d_dom_max@0 >= 1 AND d_dom_null_count@1 != row_count@2 AND d_dom_min@3 <= 2 AND (d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5 OR d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5 OR d_year_null_count@6 != row_count@2 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5), required_guarantees=[d_year in (1999, 2000, 2001)] └────────────────────────────────────────────────── "); Ok(()) @@ -7781,152 +7798,152 @@ mod tests { ┌───── 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 + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 - │ ProjectionExec: expr=[customer_id@1 as customer_id, year_total@2 as year_total, customer_id@3 as customer_id, customer_first_name@4 as customer_first_name, customer_last_name@5 as customer_last_name, year_total@6 as year_total, year_total@0 as year_total] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)], projection=[year_total@1, customer_id@2, year_total@3, customer_id@4, customer_first_name@5, customer_last_name@6, year_total@7] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ ProjectionExec: expr=[c_customer_id@0 as customer_id, c_first_name@1 as customer_first_name, c_last_name@2 as customer_last_name, sum(store_sales.ss_net_paid)@4 as year_total] - │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, d_year@3 as d_year], aggr=[sum(store_sales.ss_net_paid)], ordering_mode=PartiallySorted([3]) - │ [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 - │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(web_sales.ws_net_paid)@1 as year_total] - │ FilterExec: sum(web_sales.ws_net_paid)@1 > Some(0),17,2 - │ ProjectionExec: expr=[c_customer_id@0 as c_customer_id, sum(web_sales.ws_net_paid)@4 as sum(web_sales.ws_net_paid)] - │ 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 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 12 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[customer_id@1 as customer_id, year_total@2 as year_total, customer_id@3 as customer_id, customer_first_name@4 as customer_first_name, customer_last_name@5 as customer_last_name, year_total@6 as year_total, year_total@0 as year_total] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)], projection=[year_total@1, customer_id@2, year_total@3, customer_id@4, customer_first_name@5, customer_last_name@6, year_total@7] + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(customer_id@0, customer_id@0)] + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, input_tasks=2 + │ ProjectionExec: expr=[c_customer_id@0 as customer_id, c_first_name@1 as customer_first_name, c_last_name@2 as customer_last_name, sum(store_sales.ss_net_paid)@4 as year_total] + │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, d_year@3 as d_year], aggr=[sum(store_sales.ss_net_paid)], ordering_mode=PartiallySorted([3]) + │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(web_sales.ws_net_paid)@1 as year_total] + │ FilterExec: sum(web_sales.ws_net_paid)@1 > Some(0),17,2 + │ ProjectionExec: expr=[c_customer_id@0 as c_customer_id, sum(web_sales.ws_net_paid)@4 as sum(web_sales.ws_net_paid)] + │ 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 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, d_year@3], 6), input_partitions=2 │ AggregateExec: mode=Partial, 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@4 as d_year], aggr=[sum(web_sales.ws_net_paid)], ordering_mode=PartiallySorted([3]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, ws_net_paid@4 as ws_net_paid, 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@3)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, ws_net_paid@7] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@4, ws_bill_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, ws_sold_date_sk@5, ws_net_paid@7] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_customer_sk, ws_net_paid], 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 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001 + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001 + │ 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_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 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 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/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 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 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_customer_id, c_first_name, c_last_name], file_type=parquet + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(store_sales.ss_net_paid)@1 as year_total] - │ FilterExec: sum(store_sales.ss_net_paid)@1 > Some(0),17,2 - │ ProjectionExec: expr=[c_customer_id@0 as c_customer_id, sum(store_sales.ss_net_paid)@4 as sum(store_sales.ss_net_paid)] - │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, d_year@3 as d_year], aggr=[sum(store_sales.ss_net_paid)], ordering_mode=PartiallySorted([3]) - │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 8 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[c_customer_id@0 as customer_id, sum(store_sales.ss_net_paid)@1 as year_total] + │ FilterExec: sum(store_sales.ss_net_paid)@1 > Some(0),17,2 + │ ProjectionExec: expr=[c_customer_id@0 as c_customer_id, sum(store_sales.ss_net_paid)@4 as sum(store_sales.ss_net_paid)] + │ AggregateExec: mode=FinalPartitioned, gby=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, d_year@3 as d_year], aggr=[sum(store_sales.ss_net_paid)], ordering_mode=PartiallySorted([3]) + │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, d_year@3], 6), input_partitions=2 │ AggregateExec: mode=Partial, 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@4 as d_year], aggr=[sum(store_sales.ss_net_paid)], ordering_mode=PartiallySorted([3]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, ss_net_paid@4 as ss_net_paid, 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@3)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, ss_net_paid@7] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@4, ss_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, ss_sold_date_sk@5, ss_net_paid@7] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_customer_sk, ss_net_paid], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001 + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001 + │ 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_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..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 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/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..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 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_customer_id, c_first_name, c_last_name], file_type=parquet + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, d_year@3], 6), input_partitions=2 │ AggregateExec: mode=Partial, 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@4 as d_year], aggr=[sum(store_sales.ss_net_paid)], ordering_mode=PartiallySorted([3]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, ss_net_paid@4 as ss_net_paid, 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@3)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, ss_net_paid@7] - │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@4, ss_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, ss_sold_date_sk@5, ss_net_paid@7] - │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_customer_sk, ss_net_paid], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2002 + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2002 + │ 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_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + └────────────────────────────────────────────────── + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 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/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 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] - └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 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_customer_id, c_first_name, c_last_name], file_type=parquet + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 15 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([c_customer_id@0, c_first_name@1, c_last_name@2, d_year@3], 6), input_partitions=2 │ AggregateExec: mode=Partial, 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@4 as d_year], aggr=[sum(web_sales.ws_net_paid)], ordering_mode=PartiallySorted([3]) │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, ws_net_paid@4 as ws_net_paid, 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@3)], projection=[d_year@1, c_customer_id@3, c_first_name@4, c_last_name@5, ws_net_paid@7] - │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(customer.c_customer_sk AS Float64)@4, ws_bill_customer_sk@1)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, ws_sold_date_sk@5, ws_net_paid@7] - │ CoalescePartitionsExec - │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_customer_sk, ws_net_paid], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2002 + ┌───── Stage 13 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2002 + │ 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_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + └────────────────────────────────────────────────── + ┌───── Stage 14 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 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/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 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] - └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 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_customer_id, c_first_name, c_last_name], file_type=parquet + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_customer_id, c_first_name, c_last_name], file_type=parquet └────────────────────────────────────────────────── "); Ok(()) @@ -7940,8 +7957,8 @@ mod tests { │ SortExec: TopK(fetch=100), expr=[sales_cnt_diff@8 ASC NULLS LAST, sales_amt_diff@9 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[d_year@7 as prev_year, d_year@0 as year_, i_brand_id@1 as i_brand_id, i_class_id@2 as i_class_id, i_category_id@3 as i_category_id, i_manufact_id@4 as i_manufact_id, sales_cnt@8 as prev_yr_cnt, sales_cnt@5 as curr_yr_cnt, sales_cnt@5 - sales_cnt@8 as sales_cnt_diff, sales_amt@6 - sales_amt@9 as sales_amt_diff] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_brand_id@1, i_brand_id@1), (i_class_id@2, i_class_id@2), (i_category_id@3, i_category_id@3), (i_manufact_id@4, i_manufact_id@4)], filter=CAST(sales_cnt@0 AS Decimal128(17, 2)) / CAST(sales_cnt@1 AS Decimal128(17, 2)) < Some(900000),23,6, projection=[d_year@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@4, sales_cnt@5, sales_amt@6, d_year@7, sales_cnt@12, sales_amt@13] - │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec │ ProjectionExec: expr=[d_year@0 as d_year, i_brand_id@1 as i_brand_id, i_class_id@2 as i_class_id, i_category_id@3 as i_category_id, i_manufact_id@4 as i_manufact_id, sum(sales_detail.sales_cnt)@5 as sales_cnt, sum(sales_detail.sales_amt)@6 as sales_amt] │ AggregateExec: mode=FinalPartitioned, gby=[d_year@0 as d_year, i_brand_id@1 as i_brand_id, i_class_id@2 as i_class_id, i_category_id@3 as i_category_id, i_manufact_id@4 as i_manufact_id], aggr=[sum(sales_detail.sales_cnt), sum(sales_detail.sales_amt)], ordering_mode=PartiallySorted([0]) │ [Stage 11] => NetworkShuffleExec: output_partitions=3, input_tasks=2 @@ -7982,83 +7999,83 @@ mod tests { ┌───── 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 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2002 + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2002 + │ 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_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: i_category@4 = Books, projection=[i_item_sk@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@5] │ 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_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_category@4 = Books, projection=[i_item_sk@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@5] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_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)] + │ 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 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2002 + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2002 + │ 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_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: i_category@4 = Books, projection=[i_item_sk@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@5] │ 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_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_category@4 = Books, projection=[i_item_sk@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@5] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_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)] + │ 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 + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2002 + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2002 + │ 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_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: i_category@4 = Books, projection=[i_item_sk@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@5] │ 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_year], file_type=parquet, predicate=d_year@6 = 2002, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 2002 AND 2002 <= d_year_max@1, required_guarantees=[d_year in (2002)] - └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_category@4 = Books, projection=[i_item_sk@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@5] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_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)] + │ 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 22 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ RepartitionExec: partitioning=Hash([d_year@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@4], 3), input_partitions=3 @@ -8093,83 +8110,83 @@ mod tests { ┌───── 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 + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001 + ┌───── Stage 12 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001 + │ 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_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 13 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: i_category@4 = Books, projection=[i_item_sk@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@5] │ 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_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 13 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_category@4 = Books, projection=[i_item_sk@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@5] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_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)] + │ 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 + │ [Stage 15] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 16] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001 + ┌───── Stage 15 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001 + │ 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_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 16 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: i_category@4 = Books, projection=[i_item_sk@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@5] │ 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_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 16 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_category@4 = Books, projection=[i_item_sk@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@5] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_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)] + │ 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 + │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 19] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2001 + ┌───── Stage 18 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2001 + │ 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_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 19 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: i_category@4 = Books, projection=[i_item_sk@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@5] │ 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_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 19 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_category@4 = Books, projection=[i_item_sk@0, i_brand_id@1, i_class_id@2, i_category_id@3, i_manufact_id@5] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_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)] + │ 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)] └────────────────────────────────────────────────── "); Ok(()) @@ -8216,16 +8233,16 @@ mod tests { │ RepartitionExec: partitioning=Hash([ss_sold_date_sk@0], 3), input_partitions=2 │ ProjectionExec: expr=[ss_sold_date_sk@1 as ss_sold_date_sk, ss_ext_sales_price@2 as ss_ext_sales_price, i_category@0 as i_category] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_category@1, ss_sold_date_sk@2, ss_ext_sales_price@4] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ FilterExec: ss_store_sk@2 IS NULL, projection=[ss_sold_date_sk@0, ss_item_sk@1, ss_ext_sales_price@3] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_sales/part-2.parquet:..], ...]}, projection=[ss_sold_date_sk, ss_item_sk, ss_store_sk, ss_ext_sales_price], file_type=parquet, predicate=ss_store_sk@7 IS NULL AND DynamicFilter [ empty ] AND DynamicFilter [ empty ], pruning_predicate=ss_store_sk_null_count@0 > 0, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_category], file_type=parquet + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_category], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@3], 3), input_partitions=3 @@ -8238,16 +8255,16 @@ mod tests { │ RepartitionExec: partitioning=Hash([ws_sold_date_sk@0], 3), input_partitions=2 │ ProjectionExec: expr=[ws_sold_date_sk@1 as ws_sold_date_sk, ws_ext_sales_price@2 as ws_ext_sales_price, i_category@0 as i_category] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ws_item_sk@1)], projection=[i_category@1, ws_sold_date_sk@2, ws_ext_sales_price@4] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ FilterExec: ws_ship_customer_sk@2 IS NULL, projection=[ws_sold_date_sk@0, ws_item_sk@1, ws_ext_sales_price@3] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/web_sales/part-2.parquet:..], ...]}, projection=[ws_sold_date_sk, ws_item_sk, ws_ship_customer_sk, ws_ext_sales_price], file_type=parquet, predicate=ws_ship_customer_sk@8 IS NULL AND DynamicFilter [ empty ] AND DynamicFilter [ empty ], pruning_predicate=ws_ship_customer_sk_null_count@0 > 0, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_category], file_type=parquet + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_category], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@3], 6), input_partitions=3 @@ -8260,16 +8277,16 @@ mod tests { │ RepartitionExec: partitioning=Hash([cs_sold_date_sk@0], 6), input_partitions=2 │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_ext_sales_price@2 as cs_ext_sales_price, i_category@0 as i_category] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@1)], projection=[i_category@1, cs_sold_date_sk@2, cs_ext_sales_price@4] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ FilterExec: cs_ship_addr_sk@1 IS NULL, projection=[cs_sold_date_sk@0, cs_item_sk@2, cs_ext_sales_price@3] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_sales/part-2.parquet:..], ...]}, projection=[cs_sold_date_sk, cs_ship_addr_sk, cs_item_sk, cs_ext_sales_price], file_type=parquet, predicate=cs_ship_addr_sk@10 IS NULL AND DynamicFilter [ empty ] AND DynamicFilter [ empty ], pruning_predicate=cs_ship_addr_sk_null_count@0 > 0, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_category], file_type=parquet + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_category], file_type=parquet └────────────────────────────────────────────────── "); Ok(()) @@ -8295,8 +8312,7 @@ mod tests { │ ProjectionExec: expr=[store channel as channel, CAST(s_store_sk@2 AS Float64) as id, sales@3 as sales, CASE WHEN __common_expr_1@0 IS NOT NULL THEN __common_expr_1@0 ELSE Some(0),22,2 END as returns_, profit@4 - CASE WHEN __common_expr_2@1 IS NOT NULL THEN __common_expr_2@1 ELSE Some(0),22,2 END as profit] │ ProjectionExec: expr=[CAST(returns_@0 AS Decimal128(22, 2)) as __common_expr_1, CAST(profit_loss@1 AS Decimal128(22, 2)) as __common_expr_2, s_store_sk@2 as s_store_sk, sales@3 as sales, profit@4 as profit] │ HashJoinExec: mode=CollectLeft, join_type=Right, on=[(s_store_sk@0, s_store_sk@0)], projection=[returns_@1, profit_loss@2, s_store_sk@3, sales@4, profit@5] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, input_tasks=2 │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, sum(store_sales.ss_ext_sales_price)@1 as sales, sum(store_sales.ss_net_profit)@2 as profit] │ AggregateExec: mode=FinalPartitioned, gby=[s_store_sk@0 as s_store_sk], aggr=[sum(store_sales.ss_ext_sales_price), sum(store_sales.ss_net_profit)] │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 @@ -8316,63 +8332,64 @@ mod tests { │ 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 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, sum(store_returns.sr_return_amt)@1 as returns_, sum(store_returns.sr_net_loss)@2 as profit_loss] - │ AggregateExec: mode=FinalPartitioned, gby=[s_store_sk@0 as s_store_sk], aggr=[sum(store_returns.sr_return_amt), sum(store_returns.sr_net_loss)] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, sum(store_returns.sr_return_amt)@1 as returns_, sum(store_returns.sr_net_loss)@2 as profit_loss] + │ AggregateExec: mode=FinalPartitioned, gby=[s_store_sk@0 as s_store_sk], aggr=[sum(store_returns.sr_return_amt), sum(store_returns.sr_net_loss)] + │ [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_sk@0], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[s_store_sk@2 as s_store_sk], aggr=[sum(store_returns.sr_return_amt), sum(store_returns.sr_net_loss)] │ ProjectionExec: expr=[sr_return_amt@1 as sr_return_amt, sr_net_loss@2 as sr_net_loss, s_store_sk@0 as s_store_sk] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, sr_store_sk@0)], projection=[s_store_sk@0, sr_return_amt@3, sr_net_loss@4] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, sr_returned_date_sk@0)], projection=[sr_store_sk@3, sr_return_amt@4, sr_net_loss@5] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_returned_date_sk, sr_store_sk, sr_return_amt, sr_net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] - │ 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, 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 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] + │ 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, 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 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([s_store_sk@0], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[s_store_sk@2 as s_store_sk], aggr=[sum(store_sales.ss_ext_sales_price), sum(store_sales.ss_net_profit)] │ ProjectionExec: expr=[ss_ext_sales_price@1 as ss_ext_sales_price, ss_net_profit@2 as ss_net_profit, s_store_sk@0 as s_store_sk] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@1, ss_store_sk@0)], projection=[s_store_sk@0, ss_ext_sales_price@3, ss_net_profit@4] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_store_sk@3, ss_ext_sales_price@4, ss_net_profit@5] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_store_sk, ss_ext_sales_price, ss_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] - │ 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, 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 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] + │ 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, 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] @@ -8383,34 +8400,34 @@ mod tests { │ RepartitionExec: partitioning=Hash([cs_call_center_sk@0], 6), input_partitions=2 │ AggregateExec: mode=Partial, 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)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_call_center_sk@3, cs_ext_sales_price@4, cs_net_profit@5] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_call_center_sk, cs_ext_sales_price, cs_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] - │ 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, 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 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] + │ 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, 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 12 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ RepartitionExec: partitioning=Hash([cr_call_center_sk@0], 3), input_partitions=2 │ AggregateExec: mode=Partial, gby=[cr_call_center_sk@0 as cr_call_center_sk], aggr=[sum(catalog_returns.cr_return_amount), sum(catalog_returns.cr_net_loss)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_date_sk@0, cr_returned_date_sk@0)], projection=[cr_call_center_sk@2, cr_return_amount@3, cr_net_loss@4] - │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_returned_date_sk, cr_call_center_sk, cr_return_amount, cr_net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: d_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] - │ 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, 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 11 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: d_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] + │ 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, 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] @@ -8422,53 +8439,53 @@ mod tests { │ AggregateExec: mode=Partial, gby=[wp_web_page_sk@2 as wp_web_page_sk], aggr=[sum(web_sales.ws_ext_sales_price), sum(web_sales.ws_net_profit)] │ ProjectionExec: expr=[ws_ext_sales_price@1 as ws_ext_sales_price, ws_net_profit@2 as ws_net_profit, wp_web_page_sk@0 as wp_web_page_sk] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_page.wp_web_page_sk AS Float64)@1, ws_web_page_sk@0)], projection=[wp_web_page_sk@0, ws_ext_sales_price@3, ws_net_profit@4] - │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_web_page_sk@3, ws_ext_sales_price@4, ws_net_profit@5] - │ CoalescePartitionsExec - │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_web_page_sk, ws_ext_sales_price, ws_net_profit], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_page/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-3.parquet]]}, projection=[wp_web_page_sk, CAST(wp_web_page_sk@0 AS Float64) as CAST(web_page.wp_web_page_sk AS Float64)], file_type=parquet + ┌───── Stage 13 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_page/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-3.parquet]]}, projection=[wp_web_page_sk, CAST(wp_web_page_sk@0 AS Float64) as CAST(web_page.wp_web_page_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] - │ 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, 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 14 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] + │ 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, 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 19 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ RepartitionExec: partitioning=Hash([wp_web_page_sk@0], 3), input_partitions=3 │ AggregateExec: mode=Partial, gby=[wp_web_page_sk@2 as wp_web_page_sk], aggr=[sum(web_returns.wr_return_amt), sum(web_returns.wr_net_loss)] │ ProjectionExec: expr=[wr_return_amt@1 as wr_return_amt, wr_net_loss@2 as wr_net_loss, wp_web_page_sk@0 as wp_web_page_sk] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_page.wp_web_page_sk AS Float64)@1, wr_web_page_sk@0)], projection=[wp_web_page_sk@0, wr_return_amt@3, wr_net_loss@4] - │ CoalescePartitionsExec - │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=2 + │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, wr_returned_date_sk@0)], projection=[wr_web_page_sk@3, wr_return_amt@4, wr_net_loss@5] - │ CoalescePartitionsExec - │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 + │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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/web_returns/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_returned_date_sk, wr_web_page_sk, wr_return_amt, wr_net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 17 ── Tasks: t0:[p0..p3] t1:[p4..p7] - │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_page/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-3.parquet]]}, projection=[wp_web_page_sk, CAST(wp_web_page_sk@0 AS Float64) as CAST(web_page.wp_web_page_sk AS Float64)], file_type=parquet + ┌───── Stage 17 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_page/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-3.parquet]]}, projection=[wp_web_page_sk, CAST(wp_web_page_sk@0 AS Float64) as CAST(web_page.wp_web_page_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 18 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ 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_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] - │ 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, 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 18 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ 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_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] + │ 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, 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=[] └────────────────────────────────────────────────── "); Ok(()) @@ -8506,21 +8523,21 @@ mod tests { │ AggregateExec: mode=Partial, gby=[d_year@5 as d_year, ss_item_sk@0 as ss_item_sk, ss_customer_sk@1 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]) │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_customer_sk@2 as ss_customer_sk, ss_quantity@3 as ss_quantity, ss_wholesale_cost@4 as ss_wholesale_cost, ss_sales_price@5 as ss_sales_price, 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_customer_sk@5, ss_quantity@6, ss_wholesale_cost@7, ss_sales_price@8] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ FilterExec: sr_ticket_number@6 IS NULL, projection=[ss_sold_date_sk@0, ss_item_sk@1, ss_customer_sk@2, ss_quantity@3, ss_wholesale_cost@4, ss_sales_price@5] │ ProjectionExec: expr=[ss_sold_date_sk@1 as ss_sold_date_sk, ss_item_sk@2 as ss_item_sk, ss_customer_sk@3 as ss_customer_sk, ss_quantity@4 as ss_quantity, ss_wholesale_cost@5 as ss_wholesale_cost, ss_sales_price@6 as ss_sales_price, sr_ticket_number@0 as sr_ticket_number] │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(sr_ticket_number@1, ss_ticket_number@3), (sr_item_sk@0, ss_item_sk@1)], projection=[sr_ticket_number@1, ss_sold_date_sk@2, ss_item_sk@3, ss_customer_sk@4, ss_quantity@6, ss_wholesale_cost@7, ss_sales_price@8] │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 2000 - │ 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_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 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_year@1 as d_year, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 2000 + │ 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_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 2 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ RepartitionExec: partitioning=Hash([sr_ticket_number@1, sr_item_sk@0], 9), input_partitions=2 @@ -8552,15 +8569,15 @@ mod tests { │ FilterExec: wr_order_number@6 IS NULL, projection=[ws_sold_date_sk@0, ws_item_sk@1, ws_bill_customer_sk@2, ws_quantity@3, ws_wholesale_cost@4, ws_sales_price@5] │ ProjectionExec: expr=[ws_sold_date_sk@1 as ws_sold_date_sk, ws_item_sk@2 as ws_item_sk, ws_bill_customer_sk@3 as ws_bill_customer_sk, ws_quantity@4 as ws_quantity, ws_wholesale_cost@5 as ws_wholesale_cost, ws_sales_price@6 as ws_sales_price, wr_order_number@0 as wr_order_number] │ HashJoinExec: mode=CollectLeft, join_type=Right, on=[(wr_order_number@1, ws_order_number@3), (wr_item_sk@0, ws_item_sk@1)], projection=[wr_order_number@1, ws_sold_date_sk@2, ws_item_sk@3, ws_bill_customer_sk@4, ws_quantity@6, ws_wholesale_cost@7, ws_sales_price@8] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_customer_sk, ws_order_number, ws_quantity, ws_wholesale_cost, ws_sales_price], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_returns/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_item_sk, wr_order_number], file_type=parquet + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_returns/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_item_sk, wr_order_number], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 14 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ RepartitionExec: partitioning=Hash([d_year@0, cs_item_sk@1, cs_bill_customer_sk@2], 3), input_partitions=3 @@ -8610,58 +8627,58 @@ mod tests { │ 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 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 - │ ProjectionExec: expr=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk, s_city@3 as s_city, sum(store_sales.ss_coupon_amt)@4 as amt, sum(store_sales.ss_net_profit)@5 as profit] - │ AggregateExec: mode=FinalPartitioned, gby=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk, ss_addr_sk@2 as ss_addr_sk, s_city@3 as s_city], aggr=[sum(store_sales.ss_coupon_amt), sum(store_sales.ss_net_profit)] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 5 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk, s_city@3 as s_city, sum(store_sales.ss_coupon_amt)@4 as amt, sum(store_sales.ss_net_profit)@5 as profit] + │ AggregateExec: mode=FinalPartitioned, gby=[ss_ticket_number@0 as ss_ticket_number, ss_customer_sk@1 as ss_customer_sk, ss_addr_sk@2 as ss_addr_sk, s_city@3 as s_city], aggr=[sum(store_sales.ss_coupon_amt), sum(store_sales.ss_net_profit)] + │ [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_ticket_number@0, ss_customer_sk@1, ss_addr_sk@2, s_city@3], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[ss_ticket_number@2 as ss_ticket_number, ss_customer_sk@0 as ss_customer_sk, ss_addr_sk@1 as ss_addr_sk, s_city@5 as s_city], aggr=[sum(store_sales.ss_coupon_amt), sum(store_sales.ss_net_profit)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@1, ss_hdemo_sk@1)], projection=[ss_customer_sk@2, ss_addr_sk@4, ss_ticket_number@5, ss_coupon_amt@6, ss_net_profit@7, s_city@8] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[ss_customer_sk@1 as ss_customer_sk, ss_hdemo_sk@2 as ss_hdemo_sk, ss_addr_sk@3 as ss_addr_sk, ss_ticket_number@4 as ss_ticket_number, ss_coupon_amt@5 as ss_coupon_amt, ss_net_profit@6 as ss_net_profit, s_city@0 as s_city] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, ss_store_sk@3)], projection=[s_city@1, ss_customer_sk@3, ss_hdemo_sk@4, ss_addr_sk@5, ss_ticket_number@7, ss_coupon_amt@8, ss_net_profit@9] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_customer_sk@3, ss_hdemo_sk@4, ss_addr_sk@5, ss_store_sk@6, ss_ticket_number@7, ss_coupon_amt@8, ss_net_profit@9] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_customer_sk, ss_hdemo_sk, ss_addr_sk, ss_store_sk, ss_ticket_number, ss_coupon_amt, ss_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 - │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] - │ FilterExec: hd_dep_count@1 = 6 OR hd_vehicle_count@2 > 2, projection=[hd_demo_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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 = 6 OR hd_vehicle_count@4 > 2, 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 OR hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_max@4 > 2, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, s_city@1 as s_city, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_number_employees@1 >= 200 AND s_number_employees@1 <= 295, projection=[s_store_sk@0, s_city@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/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_number_employees, s_city], file_type=parquet, predicate=s_number_employees@6 >= 200 AND s_number_employees@6 <= 295, pruning_predicate=s_number_employees_null_count@1 != row_count@2 AND s_number_employees_max@0 >= 200 AND s_number_employees_null_count@1 != row_count@2 AND s_number_employees_min@3 <= 295, required_guarantees=[] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] + │ FilterExec: hd_dep_count@1 = 6 OR hd_vehicle_count@2 > 2, projection=[hd_demo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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 = 6 OR hd_vehicle_count@4 > 2, 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 OR hd_vehicle_count_null_count@5 != row_count@3 AND hd_vehicle_count_max@4 > 2, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, s_city@1 as s_city, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_number_employees@1 >= 200 AND s_number_employees@1 <= 295, projection=[s_store_sk@0, s_city@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/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_number_employees, s_city], file_type=parquet, predicate=s_number_employees@6 >= 200 AND s_number_employees@6 <= 295, pruning_predicate=s_number_employees_null_count@1 != row_count@2 AND s_number_employees_max@0 >= 200 AND s_number_employees_null_count@1 != row_count@2 AND s_number_employees_min@3 <= 295, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_dow@2 = 1 AND (d_year@1 = 1999 OR d_year@1 = 2000 OR d_year@1 = 2001), projection=[d_date_sk@0] - │ 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_year, d_dow], file_type=parquet, predicate=d_dow@7 = 1 AND (d_year@6 = 1999 OR d_year@6 = 2000 OR d_year@6 = 2001), pruning_predicate=d_dow_null_count@2 != row_count@3 AND d_dow_min@0 <= 1 AND 1 <= d_dow_max@1 AND (d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5 OR d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5 OR d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5), required_guarantees=[d_dow in (1), d_year in (1999, 2000, 2001)] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_dow@2 = 1 AND (d_year@1 = 1999 OR d_year@1 = 2000 OR d_year@1 = 2001), projection=[d_date_sk@0] + │ 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_year, d_dow], file_type=parquet, predicate=d_dow@7 = 1 AND (d_year@6 = 1999 OR d_year@6 = 2000 OR d_year@6 = 2001), pruning_predicate=d_dow_null_count@2 != row_count@3 AND d_dow_min@0 <= 1 AND 1 <= d_dow_max@1 AND (d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 1999 AND 1999 <= d_year_max@5 OR d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2000 AND 2000 <= d_year_max@5 OR d_year_null_count@6 != row_count@3 AND d_year_min@4 <= 2001 AND 2001 <= d_year_max@5), required_guarantees=[d_dow in (1), d_year in (1999, 2000, 2001)] └────────────────────────────────────────────────── "); Ok(()) @@ -8696,50 +8713,50 @@ mod tests { │ AggregateExec: mode=Partial, gby=[s_store_id@4 as s_store_id], aggr=[sum(store_sales.ss_ext_sales_price), sum(coalesce(store_returns.sr_return_amt,Int64(0))), sum(store_sales.ss_net_profit - coalesce(store_returns.sr_net_loss,Int64(0)))] │ ProjectionExec: expr=[CAST(sr_return_amt@2 AS Decimal128(22, 2)) as __common_expr_1, CAST(sr_net_loss@3 AS Decimal128(22, 2)) as __common_expr_2, ss_ext_sales_price@0 as ss_ext_sales_price, ss_net_profit@1 as ss_net_profit, s_store_id@4 as s_store_id] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(promotion.p_promo_sk AS Float64)@1, ss_promo_sk@0)], projection=[ss_ext_sales_price@3, ss_net_profit@4, sr_return_amt@5, sr_net_loss@6, s_store_id@7] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@0)], projection=[ss_promo_sk@2, ss_ext_sales_price@3, ss_net_profit@4, sr_return_amt@5, sr_net_loss@6, s_store_id@7] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[ss_item_sk@1 as ss_item_sk, ss_promo_sk@2 as ss_promo_sk, ss_ext_sales_price@3 as ss_ext_sales_price, ss_net_profit@4 as ss_net_profit, sr_return_amt@5 as sr_return_amt, sr_net_loss@6 as sr_net_loss, s_store_id@0 as s_store_id] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@2, ss_store_sk@1)], projection=[s_store_id@1, ss_item_sk@3, ss_promo_sk@5, ss_ext_sales_price@6, ss_net_profit@7, sr_return_amt@8, sr_net_loss@9] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ 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_store_sk@4, ss_promo_sk@5, ss_ext_sales_price@6, ss_net_profit@7, sr_return_amt@8, sr_net_loss@9] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[ss_sold_date_sk@2 as ss_sold_date_sk, ss_item_sk@3 as ss_item_sk, ss_store_sk@4 as ss_store_sk, ss_promo_sk@5 as ss_promo_sk, ss_ext_sales_price@6 as ss_ext_sales_price, ss_net_profit@7 as ss_net_profit, sr_return_amt@0 as sr_return_amt, sr_net_loss@1 as sr_net_loss] │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(sr_item_sk@0, ss_item_sk@1), (sr_ticket_number@1, ss_ticket_number@4)], projection=[sr_return_amt@2, sr_net_loss@3, ss_sold_date_sk@4, ss_item_sk@5, ss_store_sk@6, ss_promo_sk@7, ss_ext_sales_price@9, ss_net_profit@10] │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[p_promo_sk@0 as p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)] - │ FilterExec: p_channel_tv@1 = N, projection=[p_promo_sk@0] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[p_promo_sk@0 as p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)] + │ FilterExec: p_channel_tv@1 = N, projection=[p_promo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, p_channel_tv], file_type=parquet, predicate=p_channel_tv@11 = N, pruning_predicate=p_channel_tv_null_count@2 != row_count@3 AND p_channel_tv_min@0 <= N AND N <= p_channel_tv_max@1, required_guarantees=[p_channel_tv in (N)] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: i_current_price@1 > Some(5000),4,2, projection=[i_item_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, p_channel_tv], file_type=parquet, predicate=p_channel_tv@11 = N, pruning_predicate=p_channel_tv_null_count@2 != row_count@3 AND p_channel_tv_min@0 <= N AND N <= p_channel_tv_max@1, required_guarantees=[p_channel_tv in (N)] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_current_price@1 > Some(5000),4,2, projection=[i_item_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_current_price], file_type=parquet, predicate=i_current_price@5 > Some(5000),4,2, pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 > Some(5000),4,2, required_guarantees=[] + │ 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_current_price], file_type=parquet, predicate=i_current_price@5 > Some(5000),4,2, pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 > Some(5000),4,2, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_id, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] - │ 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, 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 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] + │ 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, 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 5 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ RepartitionExec: partitioning=Hash([sr_item_sk@0, sr_ticket_number@1], 9), input_partitions=2 @@ -8756,52 +8773,52 @@ mod tests { │ AggregateExec: mode=Partial, gby=[cp_catalog_page_id@4 as cp_catalog_page_id], aggr=[sum(catalog_sales.cs_ext_sales_price), sum(coalesce(catalog_returns.cr_return_amount,Int64(0))), sum(catalog_sales.cs_net_profit - coalesce(catalog_returns.cr_net_loss,Int64(0)))] │ ProjectionExec: expr=[CAST(cr_return_amount@2 AS Decimal128(22, 2)) as __common_expr_3, CAST(cr_net_loss@3 AS Decimal128(22, 2)) as __common_expr_4, cs_ext_sales_price@0 as cs_ext_sales_price, cs_net_profit@1 as cs_net_profit, cp_catalog_page_id@4 as cp_catalog_page_id] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(promotion.p_promo_sk AS Float64)@1, cs_promo_sk@0)], projection=[cs_ext_sales_price@3, cs_net_profit@4, cr_return_amount@5, cr_net_loss@6, cp_catalog_page_id@7] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cs_item_sk@0)], projection=[cs_promo_sk@2, cs_ext_sales_price@3, cs_net_profit@4, cr_return_amount@5, cr_net_loss@6, cp_catalog_page_id@7] - │ CoalescePartitionsExec - │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[cs_item_sk@1 as cs_item_sk, cs_promo_sk@2 as cs_promo_sk, cs_ext_sales_price@3 as cs_ext_sales_price, cs_net_profit@4 as cs_net_profit, cr_return_amount@5 as cr_return_amount, cr_net_loss@6 as cr_net_loss, cp_catalog_page_id@0 as cp_catalog_page_id] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(catalog_page.cp_catalog_page_sk AS Float64)@2, cs_catalog_page_sk@0)], projection=[cp_catalog_page_id@1, cs_item_sk@4, cs_promo_sk@5, cs_ext_sales_price@6, cs_net_profit@7, cr_return_amount@8, cr_net_loss@9] - │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_catalog_page_sk@3, cs_item_sk@4, cs_promo_sk@5, cs_ext_sales_price@6, cs_net_profit@7, cr_return_amount@8, cr_net_loss@9] - │ CoalescePartitionsExec - │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[cs_sold_date_sk@2 as cs_sold_date_sk, cs_catalog_page_sk@3 as cs_catalog_page_sk, cs_item_sk@4 as cs_item_sk, cs_promo_sk@5 as cs_promo_sk, cs_ext_sales_price@6 as cs_ext_sales_price, cs_net_profit@7 as cs_net_profit, cr_return_amount@0 as cr_return_amount, cr_net_loss@1 as cr_net_loss] │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(cr_item_sk@0, cs_item_sk@2), (cr_order_number@1, cs_order_number@4)], projection=[cr_return_amount@2, cr_net_loss@3, cs_sold_date_sk@4, cs_catalog_page_sk@5, cs_item_sk@6, cs_promo_sk@7, cs_ext_sales_price@9, cs_net_profit@10] │ [Stage 12] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ [Stage 13] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[p_promo_sk@0 as p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)] - │ FilterExec: p_channel_tv@1 = N, projection=[p_promo_sk@0] + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[p_promo_sk@0 as p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)] + │ FilterExec: p_channel_tv@1 = N, projection=[p_promo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, p_channel_tv], file_type=parquet, predicate=p_channel_tv@11 = N, pruning_predicate=p_channel_tv_null_count@2 != row_count@3 AND p_channel_tv_min@0 <= N AND N <= p_channel_tv_max@1, required_guarantees=[p_channel_tv in (N)] + └────────────────────────────────────────────────── + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: i_current_price@1 > Some(5000),4,2, projection=[i_item_sk@0] │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, p_channel_tv], file_type=parquet, predicate=p_channel_tv@11 = N, pruning_predicate=p_channel_tv_null_count@2 != row_count@3 AND p_channel_tv_min@0 <= N AND N <= p_channel_tv_max@1, required_guarantees=[p_channel_tv in (N)] - └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_current_price@1 > Some(5000),4,2, projection=[i_item_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_current_price], file_type=parquet, predicate=i_current_price@5 > Some(5000),4,2, pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 > Some(5000),4,2, required_guarantees=[] + │ 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_current_price], file_type=parquet, predicate=i_current_price@5 > Some(5000),4,2, pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 > Some(5000),4,2, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[cp_catalog_page_sk@0 as cp_catalog_page_sk, cp_catalog_page_id@1 as cp_catalog_page_id, CAST(cp_catalog_page_sk@0 AS Float64) as CAST(catalog_page.cp_catalog_page_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/catalog_page/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/catalog_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/catalog_page/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/catalog_page/part-3.parquet]]}, projection=[cp_catalog_page_sk, cp_catalog_page_id], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[cp_catalog_page_sk@0 as cp_catalog_page_sk, cp_catalog_page_id@1 as cp_catalog_page_id, CAST(cp_catalog_page_sk@0 AS Float64) as CAST(catalog_page.cp_catalog_page_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, 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=[] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_page/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/catalog_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/catalog_page/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/catalog_page/part-3.parquet]]}, projection=[cp_catalog_page_sk, cp_catalog_page_id], file_type=parquet + └────────────────────────────────────────────────── + ┌───── Stage 11 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] + │ 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, 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 12 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ RepartitionExec: partitioning=Hash([cr_item_sk@0, cr_order_number@1], 9), input_partitions=2 @@ -8818,50 +8835,50 @@ mod tests { │ AggregateExec: mode=Partial, gby=[web_site_id@4 as web_site_id], aggr=[sum(web_sales.ws_ext_sales_price), sum(coalesce(web_returns.wr_return_amt,Int64(0))), sum(web_sales.ws_net_profit - coalesce(web_returns.wr_net_loss,Int64(0)))] │ ProjectionExec: expr=[CAST(wr_return_amt@2 AS Decimal128(22, 2)) as __common_expr_5, CAST(wr_net_loss@3 AS Decimal128(22, 2)) as __common_expr_6, ws_ext_sales_price@0 as ws_ext_sales_price, ws_net_profit@1 as ws_net_profit, web_site_id@4 as web_site_id] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_promo_sk@0, CAST(promotion.p_promo_sk AS Float64)@1)], projection=[ws_ext_sales_price@1, ws_net_profit@2, wr_return_amt@3, wr_net_loss@4, web_site_id@5] - │ CoalescePartitionsExec - │ [Stage 20] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ [Stage 20] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[p_promo_sk@0 as p_promo_sk, CAST(p_promo_sk@0 AS Float64) as CAST(promotion.p_promo_sk AS Float64)] │ FilterExec: p_channel_tv@1 = N, projection=[p_promo_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/promotion/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/promotion/part-3.parquet]]}, projection=[p_promo_sk, p_channel_tv], file_type=parquet, predicate=p_channel_tv@11 = N, pruning_predicate=p_channel_tv_null_count@2 != row_count@3 AND p_channel_tv_min@0 <= N AND N <= p_channel_tv_max@1, required_guarantees=[p_channel_tv in (N)] └────────────────────────────────────────────────── - ┌───── Stage 20 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_item_sk@0, i_item_sk@0)], projection=[ws_promo_sk@1, ws_ext_sales_price@2, ws_net_profit@3, wr_return_amt@4, wr_net_loss@5, web_site_id@6] - │ CoalescePartitionsExec - │ [Stage 19] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ FilterExec: i_current_price@1 > Some(5000),4,2, 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_current_price], file_type=parquet, predicate=i_current_price@5 > Some(5000),4,2 AND DynamicFilter [ empty ], pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 > Some(5000),4,2, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 19 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[ws_item_sk@1 as ws_item_sk, ws_promo_sk@2 as ws_promo_sk, ws_ext_sales_price@3 as ws_ext_sales_price, ws_net_profit@4 as ws_net_profit, wr_return_amt@5 as wr_return_amt, wr_net_loss@6 as wr_net_loss, web_site_id@0 as web_site_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_site.web_site_sk AS Float64)@2, ws_web_site_sk@1)], projection=[web_site_id@1, ws_item_sk@3, ws_promo_sk@5, ws_ext_sales_price@6, ws_net_profit@7, wr_return_amt@8, wr_net_loss@9] - │ CoalescePartitionsExec - │ [Stage 15] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, 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_item_sk@3, ws_web_site_sk@4, ws_promo_sk@5, ws_ext_sales_price@6, ws_net_profit@7, wr_return_amt@8, wr_net_loss@9] - │ CoalescePartitionsExec - │ [Stage 16] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[ws_sold_date_sk@2 as ws_sold_date_sk, ws_item_sk@3 as ws_item_sk, ws_web_site_sk@4 as ws_web_site_sk, ws_promo_sk@5 as ws_promo_sk, ws_ext_sales_price@6 as ws_ext_sales_price, ws_net_profit@7 as ws_net_profit, wr_return_amt@0 as wr_return_amt, wr_net_loss@1 as wr_net_loss] - │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(wr_item_sk@0, ws_item_sk@1), (wr_order_number@1, ws_order_number@4)], projection=[wr_return_amt@2, wr_net_loss@3, ws_sold_date_sk@4, ws_item_sk@5, ws_web_site_sk@6, ws_promo_sk@7, ws_ext_sales_price@9, ws_net_profit@10] - │ [Stage 17] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 18] => NetworkShuffleExec: output_partitions=3, input_tasks=3 - └────────────────────────────────────────────────── - ┌───── Stage 15 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_site/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-3.parquet]]}, projection=[web_site_sk, web_site_id, CAST(web_site_sk@0 AS Float64) as CAST(web_site.web_site_sk AS Float64)], file_type=parquet + ┌───── Stage 20 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_item_sk@0, i_item_sk@0)], projection=[ws_promo_sk@1, ws_ext_sales_price@2, ws_net_profit@3, wr_return_amt@4, wr_net_loss@5, web_site_id@6] + │ [Stage 19] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ FilterExec: i_current_price@1 > Some(5000),4,2, 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_current_price], file_type=parquet, predicate=i_current_price@5 > Some(5000),4,2 AND DynamicFilter [ empty ], pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 > Some(5000),4,2, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 19 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ws_item_sk@1 as ws_item_sk, ws_promo_sk@2 as ws_promo_sk, ws_ext_sales_price@3 as ws_ext_sales_price, ws_net_profit@4 as ws_net_profit, wr_return_amt@5 as wr_return_amt, wr_net_loss@6 as wr_net_loss, web_site_id@0 as web_site_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_site.web_site_sk AS Float64)@2, ws_web_site_sk@1)], projection=[web_site_id@1, ws_item_sk@3, ws_promo_sk@5, ws_ext_sales_price@6, ws_net_profit@7, wr_return_amt@8, wr_net_loss@9] + │ [Stage 15] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_item_sk@3, ws_web_site_sk@4, ws_promo_sk@5, ws_ext_sales_price@6, ws_net_profit@7, wr_return_amt@8, wr_net_loss@9] + │ [Stage 16] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ ProjectionExec: expr=[ws_sold_date_sk@2 as ws_sold_date_sk, ws_item_sk@3 as ws_item_sk, ws_web_site_sk@4 as ws_web_site_sk, ws_promo_sk@5 as ws_promo_sk, ws_ext_sales_price@6 as ws_ext_sales_price, ws_net_profit@7 as ws_net_profit, wr_return_amt@0 as wr_return_amt, wr_net_loss@1 as wr_net_loss] + │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(wr_item_sk@0, ws_item_sk@1), (wr_order_number@1, ws_order_number@4)], projection=[wr_return_amt@2, wr_net_loss@3, ws_sold_date_sk@4, ws_item_sk@5, ws_web_site_sk@6, ws_promo_sk@7, ws_ext_sales_price@9, ws_net_profit@10] + │ [Stage 17] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 18] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 15 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_site/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-3.parquet]]}, projection=[web_site_sk, web_site_id, CAST(web_site_sk@0 AS Float64) as CAST(web_site.web_site_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 16 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] - │ 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, 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:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_date@1 >= 2000-08-23 AND d_date@1 <= 2000-09-22, projection=[d_date_sk@0] + │ 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, 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 17 ── Tasks: t0:[p0..p8] t1:[p0..p8] │ RepartitionExec: partitioning=Hash([wr_item_sk@0, wr_order_number@1], 9), input_partitions=2 @@ -8888,64 +8905,64 @@ mod tests { │ 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 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 - │ ProjectionExec: expr=[ctr_state@11 as ctr_state, ctr_total_return@12 as ctr_total_return, ca_street_number@0 as ca_street_number, ca_street_name@1 as ca_street_name, ca_street_type@2 as ca_street_type, ca_suite_number@3 as ca_suite_number, ca_city@4 as ca_city, ca_county@5 as ca_county, ca_state@6 as ca_state, ca_zip@7 as ca_zip, ca_country@8 as ca_country, ca_gmt_offset@9 as ca_gmt_offset, ca_location_type@10 as ca_location_type, c_customer_id@13 as c_customer_id, c_salutation@14 as c_salutation, c_first_name@15 as c_first_name, c_last_name@16 as c_last_name] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@3)], projection=[ca_street_number@1, ca_street_name@2, ca_street_type@3, ca_suite_number@4, ca_city@5, ca_county@6, ca_state@7, ca_zip@8, ca_country@9, ca_gmt_offset@10, ca_location_type@11, ctr_state@12, ctr_total_return@13, c_customer_id@14, c_salutation@16, c_first_name@17, c_last_name@18] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ctr_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@6)], projection=[ctr_state@1, ctr_total_return@2, c_customer_id@4, c_current_addr_sk@5, c_salutation@6, c_first_name@7, c_last_name@8] - │ 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_customer_id@1 as c_customer_id, c_current_addr_sk@2 as c_current_addr_sk, c_salutation@3 as c_salutation, c_first_name@4 as c_first_name, c_last_name@5 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_customer_id, c_current_addr_sk, c_salutation, c_first_name, c_last_name], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 6 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ctr_state@11 as ctr_state, ctr_total_return@12 as ctr_total_return, ca_street_number@0 as ca_street_number, ca_street_name@1 as ca_street_name, ca_street_type@2 as ca_street_type, ca_suite_number@3 as ca_suite_number, ca_city@4 as ca_city, ca_county@5 as ca_county, ca_state@6 as ca_state, ca_zip@7 as ca_zip, ca_country@8 as ca_country, ca_gmt_offset@9 as ca_gmt_offset, ca_location_type@10 as ca_location_type, c_customer_id@13 as c_customer_id, c_salutation@14 as c_salutation, c_first_name@15 as c_first_name, c_last_name@16 as c_last_name] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@3)], projection=[ca_street_number@1, ca_street_name@2, ca_street_type@3, ca_suite_number@4, ca_city@5, ca_county@6, ca_state@7, ca_zip@8, ca_country@9, ca_gmt_offset@10, ca_location_type@11, ctr_state@12, ctr_total_return@13, c_customer_id@14, c_salutation@16, c_first_name@17, c_last_name@18] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ctr_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@6)], projection=[ctr_state@1, ctr_total_return@2, c_customer_id@4, c_current_addr_sk@5, c_salutation@6, c_first_name@7, c_last_name@8] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, input_tasks=2 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_customer_id@1 as c_customer_id, c_current_addr_sk@2 as c_current_addr_sk, c_salutation@3 as c_salutation, c_first_name@4 as c_first_name, c_last_name@5 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_customer_id, c_current_addr_sk, c_salutation, c_first_name, c_last_name], 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 - │ FilterExec: ca_state@7 = GA - │ 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_street_number, ca_street_name, ca_street_type, ca_suite_number, ca_city, ca_county, ca_state, ca_zip, ca_country, ca_gmt_offset, ca_location_type], file_type=parquet, predicate=ca_state@8 = GA, pruning_predicate=ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= GA AND GA <= ca_state_max@1, required_guarantees=[ca_state in (GA)] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ FilterExec: ca_state@7 = GA + │ 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_street_number, ca_street_name, ca_street_type, ca_suite_number, ca_city, ca_county, ca_state, ca_zip, ca_country, ca_gmt_offset, ca_location_type], file_type=parquet, predicate=ca_state@8 = GA, pruning_predicate=ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= GA AND GA <= ca_state_max@1, required_guarantees=[ca_state in (GA)] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ ProjectionExec: expr=[cr_returning_customer_sk@0 as ctr_customer_sk, ca_state@1 as ctr_state, sum(catalog_returns.cr_return_amt_inc_tax)@2 as ctr_total_return] - │ AggregateExec: mode=FinalPartitioned, gby=[cr_returning_customer_sk@0 as cr_returning_customer_sk, ca_state@1 as ca_state], aggr=[sum(catalog_returns.cr_return_amt_inc_tax)] - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 5 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[cr_returning_customer_sk@0 as ctr_customer_sk, ca_state@1 as ctr_state, sum(catalog_returns.cr_return_amt_inc_tax)@2 as ctr_total_return] + │ AggregateExec: mode=FinalPartitioned, gby=[cr_returning_customer_sk@0 as cr_returning_customer_sk, ca_state@1 as ca_state], aggr=[sum(catalog_returns.cr_return_amt_inc_tax)] + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([cr_returning_customer_sk@0, ca_state@1], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[cr_returning_customer_sk@0 as cr_returning_customer_sk, ca_state@2 as ca_state], aggr=[sum(catalog_returns.cr_return_amt_inc_tax)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cr_returning_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@2)], projection=[cr_returning_customer_sk@0, cr_return_amt_inc_tax@2, ca_state@4] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_state@1 as ca_state, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_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/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_date_sk@0, cr_returned_date_sk@0)], projection=[cr_returning_customer_sk@2, cr_returning_addr_sk@3, cr_return_amt_inc_tax@4] - │ 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_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_returned_date_sk, cr_returning_customer_sk, cr_returning_addr_sk, cr_return_amt_inc_tax], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_date_sk@0, cr_returned_date_sk@0)], projection=[cr_returning_customer_sk@2, cr_returning_addr_sk@3, cr_return_amt_inc_tax@4] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_returned_date_sk, cr_returning_customer_sk, cr_returning_addr_sk, cr_return_amt_inc_tax], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: d_year@1 = 2000, projection=[d_date_sk@0] - │ 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_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 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: d_year@1 = 2000, projection=[d_date_sk@0] + │ 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_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 10 ── Tasks: t0:[p0..p5] t1:[p0..p5] │ RepartitionExec: partitioning=Hash([ctr_state@0], 6), input_partitions=3 @@ -8958,27 +8975,27 @@ mod tests { │ RepartitionExec: partitioning=Hash([cr_returning_customer_sk@0, ca_state@1], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[cr_returning_customer_sk@0 as cr_returning_customer_sk, ca_state@2 as ca_state], aggr=[sum(catalog_returns.cr_return_amt_inc_tax)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cr_returning_addr_sk@1, CAST(customer_address.ca_address_sk AS Float64)@2)], projection=[cr_returning_customer_sk@0, cr_return_amt_inc_tax@2, ca_state@4] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_state@1 as ca_state, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_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/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_date_sk@0, cr_returned_date_sk@0)], projection=[cr_returning_customer_sk@2, cr_returning_addr_sk@3, cr_return_amt_inc_tax@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/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_returned_date_sk, cr_returning_customer_sk, cr_returning_addr_sk, cr_return_amt_inc_tax], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: d_year@1 = 2000, projection=[d_date_sk@0] - │ 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_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 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_date_sk@0, cr_returned_date_sk@0)], projection=[cr_returning_customer_sk@2, cr_returning_addr_sk@3, cr_return_amt_inc_tax@4] + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_returned_date_sk, cr_returning_customer_sk, cr_returning_addr_sk, cr_return_amt_inc_tax], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: d_year@1 = 2000, projection=[d_date_sk@0] + │ 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_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)] └────────────────────────────────────────────────── "); Ok(()) @@ -9000,38 +9017,38 @@ mod tests { │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, i_current_price@2], 6), input_partitions=2 │ AggregateExec: mode=Partial, 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=[] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@0)], projection=[i_item_id@1, i_item_desc@2, i_current_price@3] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ 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], 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 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(inv_date_sk@4, d_date_sk@0)], projection=[i_item_sk@0, i_item_id@1, i_item_desc@2, i_current_price@3] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ FilterExec: d_date@1 >= 2000-05-25 AND d_date@1 <= 2000-07-24, 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_date], file_type=parquet, predicate=d_date@2 >= 2000-05-25 AND d_date@2 <= 2000-07-24 AND DynamicFilter [ empty ], pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-05-25 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-07-24, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, inv_item_sk@1)], projection=[i_item_sk@0, i_item_id@1, i_item_desc@2, i_current_price@3, inv_date_sk@4] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ FilterExec: inv_quantity_on_hand@2 >= 100 AND inv_quantity_on_hand@2 <= 500, projection=[inv_date_sk@0, inv_item_sk@1] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/inventory/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], ...]}, projection=[inv_date_sk, inv_item_sk, inv_quantity_on_hand], file_type=parquet, predicate=inv_quantity_on_hand@3 >= 100 AND inv_quantity_on_hand@3 <= 500 AND DynamicFilter [ empty ], pruning_predicate=inv_quantity_on_hand_null_count@1 != row_count@2 AND inv_quantity_on_hand_max@0 >= 100 AND inv_quantity_on_hand_null_count@1 != row_count@2 AND inv_quantity_on_hand_min@3 <= 500, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_current_price@3 >= Some(6200),4,2 AND i_current_price@3 <= Some(9200),4,2 AND i_manufact_id@4 IN (SET) ([129, 270, 821, 423]), projection=[i_item_sk@0, i_item_id@1, i_item_desc@2, i_current_price@3] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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, i_item_desc, i_current_price, i_manufact_id], file_type=parquet, predicate=i_current_price@5 >= Some(6200),4,2 AND i_current_price@5 <= Some(9200),4,2 AND i_manufact_id@13 IN (SET) ([129, 270, 821, 423]) AND DynamicFilter [ empty ], pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 >= Some(6200),4,2 AND i_current_price_null_count@1 != row_count@2 AND i_current_price_min@3 <= Some(9200),4,2 AND (i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 129 AND 129 <= i_manufact_id_max@5 OR i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 270 AND 270 <= i_manufact_id_max@5 OR i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 821 AND 821 <= i_manufact_id_max@5 OR i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 423 AND 423 <= i_manufact_id_max@5), required_guarantees=[i_manufact_id in (129, 270, 423, 821)] - └────────────────────────────────────────────────── - "); + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(inv_date_sk@4, d_date_sk@0)], projection=[i_item_sk@0, i_item_id@1, i_item_desc@2, i_current_price@3] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ FilterExec: d_date@1 >= 2000-05-25 AND d_date@1 <= 2000-07-24, 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_date], file_type=parquet, predicate=d_date@2 >= 2000-05-25 AND d_date@2 <= 2000-07-24 AND DynamicFilter [ empty ], pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-05-25 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-07-24, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, inv_item_sk@1)], projection=[i_item_sk@0, i_item_id@1, i_item_desc@2, i_current_price@3, inv_date_sk@4] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ FilterExec: inv_quantity_on_hand@2 >= 100 AND inv_quantity_on_hand@2 <= 500, projection=[inv_date_sk@0, inv_item_sk@1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/inventory/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/inventory/part-2.parquet:..], ...]}, projection=[inv_date_sk, inv_item_sk, inv_quantity_on_hand], file_type=parquet, predicate=inv_quantity_on_hand@3 >= 100 AND inv_quantity_on_hand@3 <= 500 AND DynamicFilter [ empty ], pruning_predicate=inv_quantity_on_hand_null_count@1 != row_count@2 AND inv_quantity_on_hand_max@0 >= 100 AND inv_quantity_on_hand_null_count@1 != row_count@2 AND inv_quantity_on_hand_min@3 <= 500, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: i_current_price@3 >= Some(6200),4,2 AND i_current_price@3 <= Some(9200),4,2 AND i_manufact_id@4 IN (SET) ([129, 270, 821, 423]), projection=[i_item_sk@0, i_item_id@1, i_item_desc@2, i_current_price@3] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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, i_item_desc, i_current_price, i_manufact_id], file_type=parquet, predicate=i_current_price@5 >= Some(6200),4,2 AND i_current_price@5 <= Some(9200),4,2 AND i_manufact_id@13 IN (SET) ([129, 270, 821, 423]) AND DynamicFilter [ empty ], pruning_predicate=i_current_price_null_count@1 != row_count@2 AND i_current_price_max@0 >= Some(6200),4,2 AND i_current_price_null_count@1 != row_count@2 AND i_current_price_min@3 <= Some(9200),4,2 AND (i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 129 AND 129 <= i_manufact_id_max@5 OR i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 270 AND 270 <= i_manufact_id_max@5 OR i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 821 AND 821 <= i_manufact_id_max@5 OR i_manufact_id_null_count@6 != row_count@2 AND i_manufact_id_min@4 <= 423 AND 423 <= i_manufact_id_max@5), required_guarantees=[i_manufact_id in (129, 270, 423, 821)] + └────────────────────────────────────────────────── + "); Ok(()) } #[tokio::test] @@ -9047,89 +9064,89 @@ mod tests { │ 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 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 - │ ProjectionExec: expr=[i_item_id@0 as item_id, sum(web_returns.wr_return_quantity)@1 as wr_item_qty] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(web_returns.wr_return_quantity)] - │ RepartitionExec: partitioning=Hash([i_item_id@0], 3), input_partitions=3 - │ 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 - │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d_week_seq@0, d_week_seq@1)], projection=[d_date@0] + ┌───── Stage 4 ── Tasks: t0:[p0..p1] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[i_item_id@0 as item_id, sum(web_returns.wr_return_quantity)@1 as wr_item_qty] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(web_returns.wr_return_quantity)] + │ RepartitionExec: partitioning=Hash([i_item_id@0], 3), input_partitions=3 + │ 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 - │ BroadcastExec: input_partitions=3, consumer_tasks=1, output_partitions=3 - │ FilterExec: d_date@0 = 2000-06-30 OR d_date@0 = 2000-09-27 OR d_date@0 = 2000-11-17, projection=[d_week_seq@1] - │ 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=d_date@2 = 2000-06-30 OR d_date@2 = 2000-09-27 OR d_date@2 = 2000-11-17, pruning_predicate=d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-06-30 AND 2000-06-30 <= d_date_max@1 OR d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-09-27 AND 2000-09-27 <= d_date_max@1 OR d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-11-17 AND 2000-11-17 <= d_date_max@1, required_guarantees=[d_date in (2000-06-30, 2000-09-27, 2000-11-17)] - │ 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] => NetworkCoalesceExec: output_partitions=6, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d_week_seq@0, d_week_seq@1)], projection=[d_date@0] + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec + │ FilterExec: d_date@0 = 2000-06-30 OR d_date@0 = 2000-09-27 OR d_date@0 = 2000-11-17, projection=[d_week_seq@1] + │ 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=d_date@2 = 2000-06-30 OR d_date@2 = 2000-09-27 OR d_date@2 = 2000-11-17, pruning_predicate=d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-06-30 AND 2000-06-30 <= d_date_max@1 OR d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-09-27 AND 2000-09-27 <= d_date_max@1 OR d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-11-17 AND 2000-11-17 <= d_date_max@1, required_guarantees=[d_date in (2000-06-30, 2000-09-27, 2000-11-17)] + │ 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 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 - │ ProjectionExec: expr=[wr_returned_date_sk@1 as wr_returned_date_sk, wr_return_quantity@2 as wr_return_quantity, i_item_id@0 as i_item_id] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, wr_item_sk@1)], projection=[i_item_id@1, wr_returned_date_sk@2, wr_return_quantity@4] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, 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/web_returns/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_returned_date_sk, wr_item_sk, wr_return_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] - │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 - │ 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 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[wr_returned_date_sk@1 as wr_returned_date_sk, wr_return_quantity@2 as wr_return_quantity, i_item_id@0 as i_item_id] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, wr_item_sk@1)], projection=[i_item_id@1, wr_returned_date_sk@2, wr_return_quantity@4] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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/web_returns/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_returns/part-3.parquet]]}, projection=[wr_returned_date_sk, wr_item_sk, wr_return_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ 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 11 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ ProjectionExec: expr=[i_item_id@0 as item_id, sum(store_returns.sr_return_quantity)@1 as sr_item_qty] - │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(store_returns.sr_return_quantity)] - │ [Stage 10] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 11 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[i_item_id@0 as item_id, sum(store_returns.sr_return_quantity)@1 as sr_item_qty] + │ AggregateExec: mode=FinalPartitioned, gby=[i_item_id@0 as i_item_id], aggr=[sum(store_returns.sr_return_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], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(store_returns.sr_return_quantity)] │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d_date@0, d_date@2)], projection=[sr_return_quantity@0, i_item_id@1] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[sr_return_quantity@1 as sr_return_quantity, 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, sr_returned_date_sk@0)], projection=[d_date@1, sr_return_quantity@4, i_item_id@5] │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=2 │ [Stage 9] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d_week_seq@0, d_week_seq@1)], projection=[d_date@0] - │ CoalescePartitionsExec - │ [Stage 5] => 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/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 5 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ FilterExec: d_date@0 = 2000-06-30 OR d_date@0 = 2000-09-27 OR d_date@0 = 2000-11-17, 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_date, d_week_seq], file_type=parquet, predicate=d_date@2 = 2000-06-30 OR d_date@2 = 2000-09-27 OR d_date@2 = 2000-11-17, pruning_predicate=d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-06-30 AND 2000-06-30 <= d_date_max@1 OR d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-09-27 AND 2000-09-27 <= d_date_max@1 OR d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-11-17 AND 2000-11-17 <= d_date_max@1, required_guarantees=[d_date in (2000-06-30, 2000-09-27, 2000-11-17)] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d_week_seq@0, d_week_seq@1)], projection=[d_date@0] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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/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 5 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ FilterExec: d_date@0 = 2000-06-30 OR d_date@0 = 2000-09-27 OR d_date@0 = 2000-11-17, 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_date, d_week_seq], file_type=parquet, predicate=d_date@2 = 2000-06-30 OR d_date@2 = 2000-09-27 OR d_date@2 = 2000-11-17, pruning_predicate=d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-06-30 AND 2000-06-30 <= d_date_max@1 OR d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-09-27 AND 2000-09-27 <= d_date_max@1 OR d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-11-17 AND 2000-11-17 <= d_date_max@1, required_guarantees=[d_date in (2000-06-30, 2000-09-27, 2000-11-17)] └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p0..p8] │ RepartitionExec: partitioning=Hash([CAST(date_dim.d_date_sk AS Float64)@2], 9), input_partitions=3 @@ -9142,58 +9159,58 @@ mod tests { │ RepartitionExec: partitioning=Hash([sr_returned_date_sk@0], 9), input_partitions=2 │ ProjectionExec: expr=[sr_returned_date_sk@1 as sr_returned_date_sk, sr_return_quantity@2 as sr_return_quantity, i_item_id@0 as i_item_id] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, sr_item_sk@1)], projection=[i_item_id@1, sr_returned_date_sk@2, sr_return_quantity@4] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/store_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/store_returns/part-2.parquet:..], ...]}, projection=[sr_returned_date_sk, sr_item_sk, sr_return_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 16 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([i_item_id@0], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id], aggr=[sum(catalog_returns.cr_return_quantity)] │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d_date@0, d_date@2)], projection=[cr_return_quantity@0, i_item_id@1] - │ CoalescePartitionsExec - │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[cr_return_quantity@1 as cr_return_quantity, i_item_id@2 as i_item_id, d_date@0 as d_date] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_date_sk@0, cr_returned_date_sk@0)], projection=[d_date@1, cr_return_quantity@3, i_item_id@4] - │ CoalescePartitionsExec - │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[cr_returned_date_sk@1 as cr_returned_date_sk, cr_return_quantity@2 as cr_return_quantity, i_item_id@0 as i_item_id] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, cr_item_sk@1)], projection=[i_item_id@1, cr_returned_date_sk@2, cr_return_quantity@4] - │ CoalescePartitionsExec - │ [Stage 15] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 15] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_returned_date_sk, cr_item_sk, cr_return_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 13 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d_week_seq@0, d_week_seq@1)], projection=[d_date@0] - │ CoalescePartitionsExec - │ [Stage 12] => 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/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 13 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d_week_seq@0, d_week_seq@1)], projection=[d_date@0] + │ [Stage 12] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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/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 12 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ FilterExec: d_date@0 = 2000-06-30 OR d_date@0 = 2000-09-27 OR d_date@0 = 2000-11-17, 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_date, d_week_seq], file_type=parquet, predicate=d_date@2 = 2000-06-30 OR d_date@2 = 2000-09-27 OR d_date@2 = 2000-11-17, pruning_predicate=d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-06-30 AND 2000-06-30 <= d_date_max@1 OR d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-09-27 AND 2000-09-27 <= d_date_max@1 OR d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-11-17 AND 2000-11-17 <= d_date_max@1, required_guarantees=[d_date in (2000-06-30, 2000-09-27, 2000-11-17)] + ┌───── Stage 12 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ FilterExec: d_date@0 = 2000-06-30 OR d_date@0 = 2000-09-27 OR d_date@0 = 2000-11-17, 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_date, d_week_seq], file_type=parquet, predicate=d_date@2 = 2000-06-30 OR d_date@2 = 2000-09-27 OR d_date@2 = 2000-11-17, pruning_predicate=d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-06-30 AND 2000-06-30 <= d_date_max@1 OR d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-09-27 AND 2000-09-27 <= d_date_max@1 OR d_date_null_count@2 != row_count@3 AND d_date_min@0 <= 2000-11-17 AND 2000-11-17 <= d_date_max@1, required_guarantees=[d_date in (2000-06-30, 2000-09-27, 2000-11-17)] └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/date_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/date_dim/part-3.parquet]]}, projection=[d_date_sk, d_date], file_type=parquet + ┌───── Stage 14 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 15 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id], file_type=parquet + ┌───── Stage 15 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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 └────────────────────────────────────────────────── "); Ok(()) @@ -9211,54 +9228,54 @@ mod tests { │ 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 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ ProjectionExec: expr=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, cd_demo_sk@3 as cd_demo_sk, CAST(cd_demo_sk@3 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ib_income_band_sk@0, hd_income_band_sk@4)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, cd_demo_sk@4] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, cd_demo_sk@4 as cd_demo_sk, hd_income_band_sk@0 as hd_income_band_sk] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@2, c_current_hdemo_sk@1)], projection=[hd_income_band_sk@1, c_customer_id@3, c_first_name@5, c_last_name@6, cd_demo_sk@7] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_cdemo_sk@1, CAST(customer_demographics.cd_demo_sk AS Float64)@1)], projection=[c_customer_id@0, c_current_hdemo_sk@2, c_first_name@3, c_last_name@4, cd_demo_sk@5] - │ CoalescePartitionsExec - │ [Stage 4] => 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/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[c_customer_id@0 as c_customer_id, c_first_name@1 as c_first_name, c_last_name@2 as c_last_name, cd_demo_sk@3 as cd_demo_sk, CAST(cd_demo_sk@3 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ib_income_band_sk@0, hd_income_band_sk@4)], projection=[c_customer_id@1, c_first_name@2, c_last_name@3, cd_demo_sk@4] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ ProjectionExec: expr=[c_customer_id@1 as c_customer_id, c_first_name@2 as c_first_name, c_last_name@3 as c_last_name, cd_demo_sk@4 as cd_demo_sk, hd_income_band_sk@0 as hd_income_band_sk] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(household_demographics.hd_demo_sk AS Float64)@2, c_current_hdemo_sk@1)], projection=[hd_income_band_sk@1, c_customer_id@3, c_first_name@5, c_last_name@6, cd_demo_sk@7] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_cdemo_sk@1, CAST(customer_demographics.cd_demo_sk AS Float64)@1)], projection=[c_customer_id@0, c_current_hdemo_sk@2, c_first_name@3, c_last_name@4, cd_demo_sk@5] + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: ib_lower_bound@1 >= 38128 AND ib_upper_bound@2 <= 88128, projection=[ib_income_band_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/income_band/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-3.parquet]]}, projection=[ib_income_band_sk, ib_lower_bound, ib_upper_bound], file_type=parquet, predicate=ib_lower_bound@1 >= 38128 AND ib_upper_bound@2 <= 88128, pruning_predicate=ib_lower_bound_null_count@1 != row_count@2 AND ib_lower_bound_max@0 >= 38128 AND ib_upper_bound_null_count@4 != row_count@2 AND ib_upper_bound_min@3 <= 88128, required_guarantees=[] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: ib_lower_bound@1 >= 38128 AND ib_upper_bound@2 <= 88128, projection=[ib_income_band_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/income_band/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/income_band/part-3.parquet]]}, projection=[ib_income_band_sk, ib_lower_bound, ib_upper_bound], file_type=parquet, predicate=ib_lower_bound@1 >= 38128 AND ib_upper_bound@2 <= 88128, pruning_predicate=ib_lower_bound_null_count@1 != row_count@2 AND ib_lower_bound_max@0 >= 38128 AND ib_upper_bound_null_count@4 != row_count@2 AND ib_upper_bound_min@3 <= 88128, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ 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_income_band_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_income_band_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@3)], projection=[c_customer_id@1, c_current_cdemo_sk@2, c_current_hdemo_sk@3, c_first_name@5, c_last_name@6] - │ CoalescePartitionsExec - │ [Stage 3] => 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/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_id, c_current_cdemo_sk, c_current_hdemo_sk, c_current_addr_sk, c_first_name, c_last_name], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ca_address_sk@0, c_current_addr_sk@3)], projection=[c_customer_id@1, c_current_cdemo_sk@2, c_current_hdemo_sk@3, c_first_name@5, c_last_name@6] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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/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_id, c_current_cdemo_sk, c_current_hdemo_sk, c_current_addr_sk, c_first_name, c_last_name], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ FilterExec: ca_city@1 = Edgewood, projection=[ca_address_sk@0] - │ 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=ca_city@6 = Edgewood, pruning_predicate=ca_city_null_count@2 != row_count@3 AND ca_city_min@0 <= Edgewood AND Edgewood <= ca_city_max@1, required_guarantees=[ca_city in (Edgewood)] + ┌───── Stage 3 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ FilterExec: ca_city@1 = Edgewood, projection=[ca_address_sk@0] + │ 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=ca_city@6 = Edgewood, pruning_predicate=ca_city_null_count@2 != row_count@3 AND ca_city_min@0 <= Edgewood AND Edgewood <= ca_city_max@1, required_guarantees=[ca_city in (Edgewood)] └────────────────────────────────────────────────── "); Ok(()) @@ -9282,65 +9299,65 @@ mod tests { │ AggregateExec: mode=Partial, gby=[r_reason_desc@3 as r_reason_desc], aggr=[avg(web_sales.ws_quantity), avg(web_returns.wr_refunded_cash), avg(web_returns.wr_fee)] │ ProjectionExec: expr=[ws_quantity@1 as ws_quantity, wr_fee@2 as wr_fee, wr_refunded_cash@3 as wr_refunded_cash, r_reason_desc@0 as r_reason_desc] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(reason.r_reason_sk AS Float64)@2, wr_reason_sk@1)], projection=[r_reason_desc@1, ws_quantity@3, wr_fee@5, wr_refunded_cash@6] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_sold_date_sk@0, CAST(date_dim.d_date_sk AS Float64)@1)], projection=[ws_quantity@1, wr_reason_sk@2, wr_fee@3, wr_refunded_cash@4] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/reason/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/reason/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/reason/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/reason/part-3.parquet]]}, projection=[r_reason_sk, r_reason_desc, CAST(r_reason_sk@0 AS Float64) as CAST(reason.r_reason_sk AS Float64)], file_type=parquet - └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(wr_refunded_addr_sk@3, CAST(customer_address.ca_address_sk AS Float64)@2)], filter=(ca_state@1 = IN OR ca_state@1 = OH OR ca_state@1 = NJ) AND ws_net_profit@0 >= Some(10000),7,2 AND ws_net_profit@0 <= Some(20000),7,2 OR (ca_state@1 = WI OR ca_state@1 = CT OR ca_state@1 = KY) AND ws_net_profit@0 >= Some(15000),7,2 AND ws_net_profit@0 <= Some(30000),7,2 OR (ca_state@1 = LA OR ca_state@1 = IA OR ca_state@1 = AR) AND ws_net_profit@0 >= Some(5000),7,2 AND ws_net_profit@0 <= Some(25000),7,2, projection=[ws_sold_date_sk@0, ws_quantity@1, wr_reason_sk@4, wr_fee@5, wr_refunded_cash@6] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_state@1 as ca_state, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ FilterExec: (ca_state@1 = IN OR ca_state@1 = OH OR ca_state@1 = NJ OR ca_state@1 = WI OR ca_state@1 = CT OR ca_state@1 = KY OR ca_state@1 = LA OR ca_state@1 = IA OR ca_state@1 = AR) AND ca_state@1 IN (SET) ([IN, OH, NJ, WI, CT, KY, LA, IA, AR]) AND ca_country@2 = United States, projection=[ca_address_sk@0, ca_state@1] - │ 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/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state, ca_country], file_type=parquet, predicate=(ca_state@8 = IN OR ca_state@8 = OH OR ca_state@8 = NJ OR ca_state@8 = WI OR ca_state@8 = CT OR ca_state@8 = KY OR ca_state@8 = LA OR ca_state@8 = IA OR ca_state@8 = AR) AND ca_state@8 IN (SET) ([IN, OH, NJ, WI, CT, KY, LA, IA, AR]) AND ca_country@10 = United States, pruning_predicate=(ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IN AND IN <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OH AND OH <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= NJ AND NJ <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= WI AND WI <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= CT AND CT <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= KY AND KY <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= LA AND LA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IA AND IA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= AR AND AR <= ca_state_max@1) AND (ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IN AND IN <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OH AND OH <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= NJ AND NJ <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= WI AND WI <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= CT AND CT <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= KY AND KY <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= LA AND LA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IA AND IA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= AR AND AR <= ca_state_max@1) AND ca_country_null_count@6 != row_count@3 AND ca_country_min@4 <= United States AND United States <= ca_country_max@5, required_guarantees=[ca_country in (United States), ca_state in (AR, CT, IA, IN, KY, LA, NJ, OH, WI)] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/reason/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/reason/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/reason/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/reason/part-3.parquet]]}, projection=[r_reason_sk, r_reason_desc, CAST(r_reason_sk@0 AS Float64) as CAST(reason.r_reason_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(wr_returning_cdemo_sk@4, CAST(cd2.cd_demo_sk AS Float64)@3), (cd_marital_status@8, cd_marital_status@1), (cd_education_status@9, cd_education_status@2)], projection=[ws_sold_date_sk@0, ws_quantity@1, ws_net_profit@2, wr_refunded_addr_sk@3, wr_reason_sk@5, wr_fee@6, wr_refunded_cash@7] - │ CoalescePartitionsExec - │ [Stage 6] => 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/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, cd_education_status, CAST(cd_demo_sk@0 AS Float64) as CAST(cd2.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(wr_refunded_cdemo_sk@4, CAST(cd1.cd_demo_sk AS Float64)@3)], filter=cd_marital_status@1 = M AND cd_education_status@2 = Advanced Degree AND ws_sales_price@0 >= Some(10000),5,2 AND ws_sales_price@0 <= Some(15000),5,2 OR cd_marital_status@1 = S AND cd_education_status@2 = College AND ws_sales_price@0 >= Some(5000),5,2 AND ws_sales_price@0 <= Some(10000),5,2 OR cd_marital_status@1 = W AND cd_education_status@2 = 2 yr Degree AND ws_sales_price@0 >= Some(15000),5,2 AND ws_sales_price@0 <= Some(20000),5,2, projection=[ws_sold_date_sk@0, ws_quantity@1, ws_net_profit@3, wr_refunded_addr_sk@5, wr_returning_cdemo_sk@6, wr_reason_sk@7, wr_fee@8, wr_refunded_cash@9, cd_marital_status@11, cd_education_status@12] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 - │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, CAST(cd_demo_sk@0 AS Float64) as CAST(cd1.cd_demo_sk AS Float64)] - │ FilterExec: cd_marital_status@1 = M AND cd_education_status@2 = Advanced Degree OR cd_marital_status@1 = S AND cd_education_status@2 = College OR cd_marital_status@1 = W AND cd_education_status@2 = 2 yr Degree - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_marital_status@2 = M AND cd_education_status@3 = Advanced Degree OR cd_marital_status@2 = S AND cd_education_status@3 = College OR cd_marital_status@2 = W AND cd_education_status@3 = 2 yr Degree, pruning_predicate=cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= M AND M <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= Advanced Degree AND Advanced Degree <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= S AND S <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= College AND College <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= W AND W <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= 2 yr Degree AND 2 yr Degree <= cd_education_status_max@5, required_guarantees=[cd_education_status in (2 yr Degree, Advanced Degree, College), cd_marital_status in (M, S, W)] + ┌───── Stage 8 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(wr_refunded_addr_sk@3, CAST(customer_address.ca_address_sk AS Float64)@2)], filter=(ca_state@1 = IN OR ca_state@1 = OH OR ca_state@1 = NJ) AND ws_net_profit@0 >= Some(10000),7,2 AND ws_net_profit@0 <= Some(20000),7,2 OR (ca_state@1 = WI OR ca_state@1 = CT OR ca_state@1 = KY) AND ws_net_profit@0 >= Some(15000),7,2 AND ws_net_profit@0 <= Some(30000),7,2 OR (ca_state@1 = LA OR ca_state@1 = IA OR ca_state@1 = AR) AND ws_net_profit@0 >= Some(5000),7,2 AND ws_net_profit@0 <= Some(25000),7,2, projection=[ws_sold_date_sk@0, ws_quantity@1, wr_reason_sk@4, wr_fee@5, wr_refunded_cash@6] + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, ca_state@1 as ca_state, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] + │ FilterExec: (ca_state@1 = IN OR ca_state@1 = OH OR ca_state@1 = NJ OR ca_state@1 = WI OR ca_state@1 = CT OR ca_state@1 = KY OR ca_state@1 = LA OR ca_state@1 = IA OR ca_state@1 = AR) AND ca_state@1 IN (SET) ([IN, OH, NJ, WI, CT, KY, LA, IA, AR]) AND ca_country@2 = United States, projection=[ca_address_sk@0, ca_state@1] + │ 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/customer_address/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer_address/part-3.parquet]]}, projection=[ca_address_sk, ca_state, ca_country], file_type=parquet, predicate=(ca_state@8 = IN OR ca_state@8 = OH OR ca_state@8 = NJ OR ca_state@8 = WI OR ca_state@8 = CT OR ca_state@8 = KY OR ca_state@8 = LA OR ca_state@8 = IA OR ca_state@8 = AR) AND ca_state@8 IN (SET) ([IN, OH, NJ, WI, CT, KY, LA, IA, AR]) AND ca_country@10 = United States, pruning_predicate=(ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IN AND IN <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OH AND OH <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= NJ AND NJ <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= WI AND WI <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= CT AND CT <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= KY AND KY <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= LA AND LA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IA AND IA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= AR AND AR <= ca_state_max@1) AND (ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IN AND IN <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= OH AND OH <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= NJ AND NJ <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= WI AND WI <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= CT AND CT <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= KY AND KY <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= LA AND LA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IA AND IA <= ca_state_max@1 OR ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= AR AND AR <= ca_state_max@1) AND ca_country_null_count@6 != row_count@3 AND ca_country_min@4 <= United States AND United States <= ca_country_max@5, required_guarantees=[ca_country in (United States), ca_state in (AR, CT, IA, IN, KY, LA, NJ, OH, WI)] + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(wr_returning_cdemo_sk@4, CAST(cd2.cd_demo_sk AS Float64)@3), (cd_marital_status@8, cd_marital_status@1), (cd_education_status@9, cd_education_status@2)], projection=[ws_sold_date_sk@0, ws_quantity@1, ws_net_profit@2, wr_refunded_addr_sk@3, wr_reason_sk@5, wr_fee@6, wr_refunded_cash@7] + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, cd_education_status, CAST(cd_demo_sk@0 AS Float64) as CAST(cd2.cd_demo_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(wr_refunded_cdemo_sk@4, CAST(cd1.cd_demo_sk AS Float64)@3)], filter=cd_marital_status@1 = M AND cd_education_status@2 = Advanced Degree AND ws_sales_price@0 >= Some(10000),5,2 AND ws_sales_price@0 <= Some(15000),5,2 OR cd_marital_status@1 = S AND cd_education_status@2 = College AND ws_sales_price@0 >= Some(5000),5,2 AND ws_sales_price@0 <= Some(10000),5,2 OR cd_marital_status@1 = W AND cd_education_status@2 = 2 yr Degree AND ws_sales_price@0 >= Some(15000),5,2 AND ws_sales_price@0 <= Some(20000),5,2, projection=[ws_sold_date_sk@0, ws_quantity@1, ws_net_profit@3, wr_refunded_addr_sk@5, wr_returning_cdemo_sk@6, wr_reason_sk@7, wr_fee@8, wr_refunded_cash@9, cd_marital_status@11, cd_education_status@12] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, CAST(cd_demo_sk@0 AS Float64) as CAST(cd1.cd_demo_sk AS Float64)] + │ FilterExec: cd_marital_status@1 = M AND cd_education_status@2 = Advanced Degree OR cd_marital_status@1 = S AND cd_education_status@2 = College OR cd_marital_status@1 = W AND cd_education_status@2 = 2 yr Degree + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_marital_status@2 = M AND cd_education_status@3 = Advanced Degree OR cd_marital_status@2 = S AND cd_education_status@3 = College OR cd_marital_status@2 = W AND cd_education_status@3 = 2 yr Degree, pruning_predicate=cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= M AND M <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= Advanced Degree AND Advanced Degree <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= S AND S <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= College AND College <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= W AND W <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= 2 yr Degree AND 2 yr Degree <= cd_education_status_max@5, required_guarantees=[cd_education_status in (2 yr Degree, Advanced Degree, College), cd_marital_status in (M, S, W)] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_page.wp_web_page_sk AS Float64)@1, ws_web_page_sk@1)], projection=[ws_sold_date_sk@2, ws_quantity@4, ws_sales_price@5, ws_net_profit@6, wr_refunded_cdemo_sk@7, wr_refunded_addr_sk@8, wr_returning_cdemo_sk@9, wr_reason_sk@10, wr_fee@11, wr_refunded_cash@12] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=4, input_tasks=2 - │ ProjectionExec: expr=[ws_sold_date_sk@6 as ws_sold_date_sk, ws_web_page_sk@7 as ws_web_page_sk, ws_quantity@8 as ws_quantity, ws_sales_price@9 as ws_sales_price, ws_net_profit@10 as ws_net_profit, wr_refunded_cdemo_sk@0 as wr_refunded_cdemo_sk, wr_refunded_addr_sk@1 as wr_refunded_addr_sk, wr_returning_cdemo_sk@2 as wr_returning_cdemo_sk, wr_reason_sk@3 as wr_reason_sk, wr_fee@4 as wr_fee, wr_refunded_cash@5 as wr_refunded_cash] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(wr_item_sk@0, ws_item_sk@1), (wr_order_number@5, ws_order_number@3)], projection=[wr_refunded_cdemo_sk@1, wr_refunded_addr_sk@2, wr_returning_cdemo_sk@3, wr_reason_sk@4, wr_fee@6, wr_refunded_cash@7, ws_sold_date_sk@8, ws_web_page_sk@10, ws_quantity@12, ws_sales_price@13, ws_net_profit@14] - │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 - │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(web_page.wp_web_page_sk AS Float64)@1, ws_web_page_sk@1)], projection=[ws_sold_date_sk@2, ws_quantity@4, ws_sales_price@5, ws_net_profit@6, wr_refunded_cdemo_sk@7, wr_refunded_addr_sk@8, wr_returning_cdemo_sk@9, wr_reason_sk@10, wr_fee@11, wr_refunded_cash@12] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, input_tasks=2 + │ ProjectionExec: expr=[ws_sold_date_sk@6 as ws_sold_date_sk, ws_web_page_sk@7 as ws_web_page_sk, ws_quantity@8 as ws_quantity, ws_sales_price@9 as ws_sales_price, ws_net_profit@10 as ws_net_profit, wr_refunded_cdemo_sk@0 as wr_refunded_cdemo_sk, wr_refunded_addr_sk@1 as wr_refunded_addr_sk, wr_returning_cdemo_sk@2 as wr_returning_cdemo_sk, wr_reason_sk@3 as wr_reason_sk, wr_fee@4 as wr_fee, wr_refunded_cash@5 as wr_refunded_cash] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(wr_item_sk@0, ws_item_sk@1), (wr_order_number@5, ws_order_number@3)], projection=[wr_refunded_cdemo_sk@1, wr_refunded_addr_sk@2, wr_returning_cdemo_sk@3, wr_reason_sk@4, wr_fee@6, wr_refunded_cash@7, ws_sold_date_sk@8, ws_web_page_sk@10, ws_quantity@12, ws_sales_price@13, ws_net_profit@14] + │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=2 + │ [Stage 4] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p4..p7] - │ BroadcastExec: input_partitions=2, consumer_tasks=2, output_partitions=4 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_page/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-3.parquet]]}, projection=[wp_web_page_sk, CAST(wp_web_page_sk@0 AS Float64) as CAST(web_page.wp_web_page_sk AS Float64)], file_type=parquet + ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_page/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-3.parquet]]}, projection=[wp_web_page_sk, CAST(wp_web_page_sk@0 AS Float64) as CAST(web_page.wp_web_page_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] │ RepartitionExec: partitioning=Hash([wr_item_sk@0, wr_order_number@5], 6), input_partitions=2 @@ -9377,112 +9394,112 @@ mod tests { │ 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 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=2, 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 - │ 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 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ 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 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1, d_date@2], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] │ ProjectionExec: expr=[c_last_name@2 as c_last_name, c_first_name@1 as c_first_name, d_date@0 as d_date] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ws_bill_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@3)], projection=[d_date@1, c_first_name@3, c_last_name@4] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ 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,__] t2:[__,__,__,p0] │ 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 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ ProjectionExec: expr=[ws_bill_customer_sk@1 as ws_bill_customer_sk, d_date@0 as d_date] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ws_sold_date_sk@0)], projection=[d_date@1, ws_bill_customer_sk@4] - │ CoalescePartitionsExec - │ [Stage 1] => 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_bill_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ws_bill_customer_sk@1 as ws_bill_customer_sk, d_date@0 as d_date] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ws_sold_date_sk@0)], projection=[d_date@1, ws_bill_customer_sk@4] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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)] - │ FilterExec: d_month_seq@2 >= 1200 AND d_month_seq@2 <= 1211, projection=[d_date_sk@0, d_date@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_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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)] + │ FilterExec: d_month_seq@2 >= 1200 AND d_month_seq@2 <= 1211, projection=[d_date_sk@0, d_date@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_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=3, consumer_tasks=2, output_partitions=6 - │ 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 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 8 ── Tasks: t0:[p0..p1] t1:[p2..p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=2, output_partitions=2 + │ CoalescePartitionsExec + │ 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 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1, d_date@2], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] │ ProjectionExec: expr=[c_last_name@2 as c_last_name, c_first_name@1 as c_first_name, d_date@0 as d_date] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_bill_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@3)], projection=[d_date@1, c_first_name@3, c_last_name@4] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ 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,__] t2:[__,__,__,p0] │ 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..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ ProjectionExec: expr=[cs_bill_customer_sk@1 as cs_bill_customer_sk, d_date@0 as d_date] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, cs_sold_date_sk@0)], projection=[d_date@1, cs_bill_customer_sk@4] - │ 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_bill_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[cs_bill_customer_sk@1 as cs_bill_customer_sk, d_date@0 as d_date] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, cs_sold_date_sk@0)], projection=[d_date@1, cs_bill_customer_sk@4] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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)] - │ FilterExec: d_month_seq@2 >= 1200 AND d_month_seq@2 <= 1211, projection=[d_date_sk@0, d_date@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_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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)] + │ FilterExec: d_month_seq@2 >= 1200 AND d_month_seq@2 <= 1211, projection=[d_date_sk@0, d_date@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_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── ┌───── Stage 11 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([c_last_name@0, c_first_name@1, d_date@2], 6), input_partitions=3 │ AggregateExec: mode=Partial, gby=[c_last_name@0 as c_last_name, c_first_name@1 as c_first_name, d_date@2 as d_date], aggr=[] │ ProjectionExec: expr=[c_last_name@2 as c_last_name, c_first_name@1 as c_first_name, d_date@0 as d_date] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ss_customer_sk@0, CAST(customer.c_customer_sk AS Float64)@3)], projection=[d_date@1, c_first_name@3, c_last_name@4] - │ CoalescePartitionsExec - │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ 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,__] t2:[__,__,__,p0] │ 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 10 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ ProjectionExec: expr=[ss_customer_sk@1 as ss_customer_sk, d_date@0 as d_date] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_date@1, ss_customer_sk@4] - │ CoalescePartitionsExec - │ [Stage 9] => 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_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ss_customer_sk@1 as ss_customer_sk, d_date@0 as d_date] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@0)], projection=[d_date@1, ss_customer_sk@4] + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 9 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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)] - │ FilterExec: d_month_seq@2 >= 1200 AND d_month_seq@2 <= 1211, projection=[d_date_sk@0, d_date@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_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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)] + │ FilterExec: d_month_seq@2 >= 1200 AND d_month_seq@2 <= 1211, projection=[d_date_sk@0, d_date@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_date_sk, d_date, d_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── "); Ok(()) @@ -9542,313 +9559,313 @@ mod tests { │ 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 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_store_name@1 = ese, projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_store_name@1 = ese, projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] - │ FilterExec: t_hour@1 = 12 AND t_minute@2 < 30, projection=[t_time_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute], file_type=parquet, predicate=t_hour@3 = 12 AND t_minute@4 < 30, pruning_predicate=t_hour_null_count@2 != row_count@3 AND t_hour_min@0 <= 12 AND 12 <= t_hour_max@1 AND t_minute_null_count@5 != row_count@3 AND t_minute_min@4 < 30, required_guarantees=[t_hour in (12)] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] + │ FilterExec: t_hour@1 = 12 AND t_minute@2 < 30, projection=[t_time_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute], file_type=parquet, predicate=t_hour@3 = 12 AND t_minute@4 < 30, pruning_predicate=t_hour_null_count@2 != row_count@3 AND t_hour_min@0 <= 12 AND 12 <= t_hour_max@1 AND t_minute_null_count@5 != row_count@3 AND t_minute_min@4 < 30, required_guarantees=[t_hour in (12)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] - │ FilterExec: hd_dep_count@1 = 4 AND hd_vehicle_count@2 <= 6 OR hd_dep_count@1 = 2 AND hd_vehicle_count@2 <= 4 OR hd_dep_count@1 = 0 AND hd_vehicle_count@2 <= 2, projection=[hd_demo_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] + │ FilterExec: hd_dep_count@1 = 4 AND hd_vehicle_count@2 <= 6 OR hd_dep_count@1 = 2 AND hd_vehicle_count@2 <= 4 OR hd_dep_count@1 = 0 AND hd_vehicle_count@2 <= 2, projection=[hd_demo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_store_name@1 = ese, projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_store_name@1 = ese, projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] - │ FilterExec: t_hour@1 = 11 AND t_minute@2 >= 30, projection=[t_time_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute], file_type=parquet, predicate=t_hour@3 = 11 AND t_minute@4 >= 30, pruning_predicate=t_hour_null_count@2 != row_count@3 AND t_hour_min@0 <= 11 AND 11 <= t_hour_max@1 AND t_minute_null_count@5 != row_count@3 AND t_minute_max@4 >= 30, required_guarantees=[t_hour in (11)] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] + │ FilterExec: t_hour@1 = 11 AND t_minute@2 >= 30, projection=[t_time_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute], file_type=parquet, predicate=t_hour@3 = 11 AND t_minute@4 >= 30, pruning_predicate=t_hour_null_count@2 != row_count@3 AND t_hour_min@0 <= 11 AND 11 <= t_hour_max@1 AND t_minute_null_count@5 != row_count@3 AND t_minute_max@4 >= 30, required_guarantees=[t_hour in (11)] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] - │ FilterExec: hd_dep_count@1 = 4 AND hd_vehicle_count@2 <= 6 OR hd_dep_count@1 = 2 AND hd_vehicle_count@2 <= 4 OR hd_dep_count@1 = 0 AND hd_vehicle_count@2 <= 2, projection=[hd_demo_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] + │ FilterExec: hd_dep_count@1 = 4 AND hd_vehicle_count@2 <= 6 OR hd_dep_count@1 = 2 AND hd_vehicle_count@2 <= 4 OR hd_dep_count@1 = 0 AND hd_vehicle_count@2 <= 2, projection=[hd_demo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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 + │ [Stage 9] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 10] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 10] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 11] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_store_name@1 = ese, projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] + ┌───── Stage 9 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_store_name@1 = ese, projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] └────────────────────────────────────────────────── - ┌───── Stage 10 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] - │ FilterExec: t_hour@1 = 11 AND t_minute@2 < 30, projection=[t_time_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute], file_type=parquet, predicate=t_hour@3 = 11 AND t_minute@4 < 30, pruning_predicate=t_hour_null_count@2 != row_count@3 AND t_hour_min@0 <= 11 AND 11 <= t_hour_max@1 AND t_minute_null_count@5 != row_count@3 AND t_minute_min@4 < 30, required_guarantees=[t_hour in (11)] + ┌───── Stage 10 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] + │ FilterExec: t_hour@1 = 11 AND t_minute@2 < 30, projection=[t_time_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute], file_type=parquet, predicate=t_hour@3 = 11 AND t_minute@4 < 30, pruning_predicate=t_hour_null_count@2 != row_count@3 AND t_hour_min@0 <= 11 AND 11 <= t_hour_max@1 AND t_minute_null_count@5 != row_count@3 AND t_minute_min@4 < 30, required_guarantees=[t_hour in (11)] └────────────────────────────────────────────────── - ┌───── Stage 11 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] - │ FilterExec: hd_dep_count@1 = 4 AND hd_vehicle_count@2 <= 6 OR hd_dep_count@1 = 2 AND hd_vehicle_count@2 <= 4 OR hd_dep_count@1 = 0 AND hd_vehicle_count@2 <= 2, projection=[hd_demo_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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 11 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] + │ FilterExec: hd_dep_count@1 = 4 AND hd_vehicle_count@2 <= 6 OR hd_dep_count@1 = 2 AND hd_vehicle_count@2 <= 4 OR hd_dep_count@1 = 0 AND hd_vehicle_count@2 <= 2, projection=[hd_demo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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 + │ [Stage 13] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 14] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 14] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 15] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_store_name@1 = ese, projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] + ┌───── Stage 13 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_store_name@1 = ese, projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] └────────────────────────────────────────────────── - ┌───── Stage 14 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] - │ FilterExec: t_hour@1 = 10 AND t_minute@2 >= 30, projection=[t_time_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute], file_type=parquet, predicate=t_hour@3 = 10 AND t_minute@4 >= 30, pruning_predicate=t_hour_null_count@2 != row_count@3 AND t_hour_min@0 <= 10 AND 10 <= t_hour_max@1 AND t_minute_null_count@5 != row_count@3 AND t_minute_max@4 >= 30, required_guarantees=[t_hour in (10)] + ┌───── Stage 14 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] + │ FilterExec: t_hour@1 = 10 AND t_minute@2 >= 30, projection=[t_time_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute], file_type=parquet, predicate=t_hour@3 = 10 AND t_minute@4 >= 30, pruning_predicate=t_hour_null_count@2 != row_count@3 AND t_hour_min@0 <= 10 AND 10 <= t_hour_max@1 AND t_minute_null_count@5 != row_count@3 AND t_minute_max@4 >= 30, required_guarantees=[t_hour in (10)] └────────────────────────────────────────────────── - ┌───── Stage 15 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] - │ FilterExec: hd_dep_count@1 = 4 AND hd_vehicle_count@2 <= 6 OR hd_dep_count@1 = 2 AND hd_vehicle_count@2 <= 4 OR hd_dep_count@1 = 0 AND hd_vehicle_count@2 <= 2, projection=[hd_demo_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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 15 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] + │ FilterExec: hd_dep_count@1 = 4 AND hd_vehicle_count@2 <= 6 OR hd_dep_count@1 = 2 AND hd_vehicle_count@2 <= 4 OR hd_dep_count@1 = 0 AND hd_vehicle_count@2 <= 2, projection=[hd_demo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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 + │ [Stage 17] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 18] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 18] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 19] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_store_name@1 = ese, projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] + ┌───── Stage 17 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_store_name@1 = ese, projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] └────────────────────────────────────────────────── - ┌───── Stage 18 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] - │ FilterExec: t_hour@1 = 10 AND t_minute@2 < 30, projection=[t_time_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute], file_type=parquet, predicate=t_hour@3 = 10 AND t_minute@4 < 30, pruning_predicate=t_hour_null_count@2 != row_count@3 AND t_hour_min@0 <= 10 AND 10 <= t_hour_max@1 AND t_minute_null_count@5 != row_count@3 AND t_minute_min@4 < 30, required_guarantees=[t_hour in (10)] + ┌───── Stage 18 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] + │ FilterExec: t_hour@1 = 10 AND t_minute@2 < 30, projection=[t_time_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute], file_type=parquet, predicate=t_hour@3 = 10 AND t_minute@4 < 30, pruning_predicate=t_hour_null_count@2 != row_count@3 AND t_hour_min@0 <= 10 AND 10 <= t_hour_max@1 AND t_minute_null_count@5 != row_count@3 AND t_minute_min@4 < 30, required_guarantees=[t_hour in (10)] └────────────────────────────────────────────────── - ┌───── Stage 19 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] - │ FilterExec: hd_dep_count@1 = 4 AND hd_vehicle_count@2 <= 6 OR hd_dep_count@1 = 2 AND hd_vehicle_count@2 <= 4 OR hd_dep_count@1 = 0 AND hd_vehicle_count@2 <= 2, projection=[hd_demo_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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 19 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] + │ FilterExec: hd_dep_count@1 = 4 AND hd_vehicle_count@2 <= 6 OR hd_dep_count@1 = 2 AND hd_vehicle_count@2 <= 4 OR hd_dep_count@1 = 0 AND hd_vehicle_count@2 <= 2, projection=[hd_demo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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 + │ [Stage 21] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 22] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 22] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 23] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_store_name@1 = ese, projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] + ┌───── Stage 21 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_store_name@1 = ese, projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] └────────────────────────────────────────────────── - ┌───── Stage 22 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] - │ FilterExec: t_hour@1 = 9 AND t_minute@2 >= 30, projection=[t_time_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute], file_type=parquet, predicate=t_hour@3 = 9 AND t_minute@4 >= 30, pruning_predicate=t_hour_null_count@2 != row_count@3 AND t_hour_min@0 <= 9 AND 9 <= t_hour_max@1 AND t_minute_null_count@5 != row_count@3 AND t_minute_max@4 >= 30, required_guarantees=[t_hour in (9)] + ┌───── Stage 22 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] + │ FilterExec: t_hour@1 = 9 AND t_minute@2 >= 30, projection=[t_time_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute], file_type=parquet, predicate=t_hour@3 = 9 AND t_minute@4 >= 30, pruning_predicate=t_hour_null_count@2 != row_count@3 AND t_hour_min@0 <= 9 AND 9 <= t_hour_max@1 AND t_minute_null_count@5 != row_count@3 AND t_minute_max@4 >= 30, required_guarantees=[t_hour in (9)] └────────────────────────────────────────────────── - ┌───── Stage 23 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] - │ FilterExec: hd_dep_count@1 = 4 AND hd_vehicle_count@2 <= 6 OR hd_dep_count@1 = 2 AND hd_vehicle_count@2 <= 4 OR hd_dep_count@1 = 0 AND hd_vehicle_count@2 <= 2, projection=[hd_demo_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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 23 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] + │ FilterExec: hd_dep_count@1 = 4 AND hd_vehicle_count@2 <= 6 OR hd_dep_count@1 = 2 AND hd_vehicle_count@2 <= 4 OR hd_dep_count@1 = 0 AND hd_vehicle_count@2 <= 2, projection=[hd_demo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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 + │ [Stage 25] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 26] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 26] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 27] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_store_name@1 = ese, projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] + ┌───── Stage 25 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_store_name@1 = ese, projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] └────────────────────────────────────────────────── - ┌───── Stage 26 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] - │ FilterExec: t_hour@1 = 8 AND t_minute@2 >= 30, projection=[t_time_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute], file_type=parquet, predicate=t_hour@3 = 8 AND t_minute@4 >= 30, pruning_predicate=t_hour_null_count@2 != row_count@3 AND t_hour_min@0 <= 8 AND 8 <= t_hour_max@1 AND t_minute_null_count@5 != row_count@3 AND t_minute_max@4 >= 30, required_guarantees=[t_hour in (8)] + ┌───── Stage 26 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] + │ FilterExec: t_hour@1 = 8 AND t_minute@2 >= 30, projection=[t_time_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute], file_type=parquet, predicate=t_hour@3 = 8 AND t_minute@4 >= 30, pruning_predicate=t_hour_null_count@2 != row_count@3 AND t_hour_min@0 <= 8 AND 8 <= t_hour_max@1 AND t_minute_null_count@5 != row_count@3 AND t_minute_max@4 >= 30, required_guarantees=[t_hour in (8)] └────────────────────────────────────────────────── - ┌───── Stage 27 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] - │ FilterExec: hd_dep_count@1 = 4 AND hd_vehicle_count@2 <= 6 OR hd_dep_count@1 = 2 AND hd_vehicle_count@2 <= 4 OR hd_dep_count@1 = 0 AND hd_vehicle_count@2 <= 2, projection=[hd_demo_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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 27 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] + │ FilterExec: hd_dep_count@1 = 4 AND hd_vehicle_count@2 <= 6 OR hd_dep_count@1 = 2 AND hd_vehicle_count@2 <= 4 OR hd_dep_count@1 = 0 AND hd_vehicle_count@2 <= 2, projection=[hd_demo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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 + │ [Stage 29] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 30] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 30] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 31] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_store_name@1 = ese, projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] + ┌───── Stage 29 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_store_name@1 = ese, projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] └────────────────────────────────────────────────── - ┌───── Stage 30 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] - │ FilterExec: t_hour@1 = 9 AND t_minute@2 < 30, projection=[t_time_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute], file_type=parquet, predicate=t_hour@3 = 9 AND t_minute@4 < 30, pruning_predicate=t_hour_null_count@2 != row_count@3 AND t_hour_min@0 <= 9 AND 9 <= t_hour_max@1 AND t_minute_null_count@5 != row_count@3 AND t_minute_min@4 < 30, required_guarantees=[t_hour in (9)] + ┌───── Stage 30 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] + │ FilterExec: t_hour@1 = 9 AND t_minute@2 < 30, projection=[t_time_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute], file_type=parquet, predicate=t_hour@3 = 9 AND t_minute@4 < 30, pruning_predicate=t_hour_null_count@2 != row_count@3 AND t_hour_min@0 <= 9 AND 9 <= t_hour_max@1 AND t_minute_null_count@5 != row_count@3 AND t_minute_min@4 < 30, required_guarantees=[t_hour in (9)] └────────────────────────────────────────────────── - ┌───── Stage 31 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] - │ FilterExec: hd_dep_count@1 = 4 AND hd_vehicle_count@2 <= 6 OR hd_dep_count@1 = 2 AND hd_vehicle_count@2 <= 4 OR hd_dep_count@1 = 0 AND hd_vehicle_count@2 <= 2, projection=[hd_demo_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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 31 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] + │ FilterExec: hd_dep_count@1 = 4 AND hd_vehicle_count@2 <= 6 OR hd_dep_count@1 = 2 AND hd_vehicle_count@2 <= 4 OR hd_dep_count@1 = 0 AND hd_vehicle_count@2 <= 2, projection=[hd_demo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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)] └────────────────────────────────────────────────── "); Ok(()) @@ -9876,37 +9893,37 @@ mod tests { │ AggregateExec: mode=Partial, gby=[i_category@2 as i_category, i_class@1 as i_class, i_brand@0 as i_brand, s_store_name@5 as s_store_name, s_company_name@6 as s_company_name, d_moy@4 as d_moy], aggr=[sum(store_sales.ss_sales_price)] │ ProjectionExec: expr=[i_brand@2 as i_brand, i_class@3 as i_class, i_category@4 as i_category, ss_sales_price@5 as ss_sales_price, d_moy@6 as d_moy, s_store_name@0 as s_store_name, s_company_name@1 as s_company_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(store.s_store_sk AS Float64)@3, ss_store_sk@3)], projection=[s_store_name@1, s_company_name@2, i_brand@4, i_class@5, i_category@6, ss_sales_price@8, d_moy@9] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[i_brand@1 as i_brand, i_class@2 as i_class, i_category@3 as i_category, ss_store_sk@4 as ss_store_sk, ss_sales_price@5 as ss_sales_price, d_moy@0 as d_moy] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@2, ss_sold_date_sk@3)], projection=[d_moy@1, i_brand@3, i_class@4, i_category@5, ss_store_sk@7, ss_sales_price@8] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_brand@1, i_class@2, i_category@3, ss_sold_date_sk@4, ss_store_sk@6, ss_sales_price@7] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_store_sk, ss_sales_price], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_company_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name, s_company_name, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_moy@1 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] - │ FilterExec: d_year@1 = 1999, projection=[d_date_sk@0, d_moy@2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[d_date_sk@0 as d_date_sk, d_moy@1 as d_moy, CAST(d_date_sk@0 AS Float64) as CAST(date_dim.d_date_sk AS Float64)] + │ FilterExec: d_year@1 = 1999, projection=[d_date_sk@0, d_moy@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/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 = 1999, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1, required_guarantees=[d_year in (1999)] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: (i_category@3 = Books OR i_category@3 = Electronics OR i_category@3 = Sports) AND (i_class@2 = computers OR i_class@2 = stereo OR i_class@2 = football) OR (i_category@3 = Men OR i_category@3 = Jewelry OR i_category@3 = Women) AND (i_class@2 = shirts OR i_class@2 = birdal OR i_class@2 = dresses) │ 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_year, d_moy], file_type=parquet, predicate=d_year@6 = 1999, pruning_predicate=d_year_null_count@2 != row_count@3 AND d_year_min@0 <= 1999 AND 1999 <= d_year_max@1, required_guarantees=[d_year in (1999)] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: (i_category@3 = Books OR i_category@3 = Electronics OR i_category@3 = Sports) AND (i_class@2 = computers OR i_class@2 = stereo OR i_class@2 = football) OR (i_category@3 = Men OR i_category@3 = Jewelry OR i_category@3 = Women) AND (i_class@2 = shirts OR i_class@2 = birdal OR i_class@2 = dresses) - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_class, i_category], file_type=parquet, predicate=(i_category@12 = Books OR i_category@12 = Electronics OR i_category@12 = Sports) AND (i_class@10 = computers OR i_class@10 = stereo OR i_class@10 = football) OR (i_category@12 = Men OR i_category@12 = Jewelry OR i_category@12 = Women) AND (i_class@10 = shirts OR i_class@10 = birdal OR i_class@10 = dresses), pruning_predicate=(i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Electronics AND Electronics <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Sports AND Sports <= i_category_max@1) AND (i_class_null_count@6 != row_count@3 AND i_class_min@4 <= computers AND computers <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= stereo AND stereo <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= football AND football <= i_class_max@5) OR (i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Men AND Men <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Jewelry AND Jewelry <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Women AND Women <= i_category_max@1) AND (i_class_null_count@6 != row_count@3 AND i_class_min@4 <= shirts AND shirts <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= birdal AND birdal <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= dresses AND dresses <= i_class_max@5), required_guarantees=[] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_brand, i_class, i_category], file_type=parquet, predicate=(i_category@12 = Books OR i_category@12 = Electronics OR i_category@12 = Sports) AND (i_class@10 = computers OR i_class@10 = stereo OR i_class@10 = football) OR (i_category@12 = Men OR i_category@12 = Jewelry OR i_category@12 = Women) AND (i_class@10 = shirts OR i_class@10 = birdal OR i_class@10 = dresses), pruning_predicate=(i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Electronics AND Electronics <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Sports AND Sports <= i_category_max@1) AND (i_class_null_count@6 != row_count@3 AND i_class_min@4 <= computers AND computers <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= stereo AND stereo <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= football AND football <= i_class_max@5) OR (i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Men AND Men <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Jewelry AND Jewelry <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Women AND Women <= i_category_max@1) AND (i_class_null_count@6 != row_count@3 AND i_class_min@4 <= shirts AND shirts <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= birdal AND birdal <= i_class_max@5 OR i_class_null_count@6 != row_count@3 AND i_class_min@4 <= dresses AND dresses <= i_class_max@5), required_guarantees=[] └────────────────────────────────────────────────── "#); Ok(()) @@ -9932,79 +9949,79 @@ mod tests { │ 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 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ ProjectionExec: expr=[wp_web_page_sk@0 as wp_web_page_sk, CAST(wp_web_page_sk@0 AS Float64) as CAST(web_page.wp_web_page_sk AS Float64)] - │ FilterExec: wp_char_count@1 >= 5000 AND wp_char_count@1 <= 5200, projection=[wp_web_page_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_page/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-3.parquet]]}, projection=[wp_web_page_sk, wp_char_count], file_type=parquet, predicate=wp_char_count@10 >= 5000 AND wp_char_count@10 <= 5200, pruning_predicate=wp_char_count_null_count@1 != row_count@2 AND wp_char_count_max@0 >= 5000 AND wp_char_count_null_count@1 != row_count@2 AND wp_char_count_min@3 <= 5200, required_guarantees=[] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[wp_web_page_sk@0 as wp_web_page_sk, CAST(wp_web_page_sk@0 AS Float64) as CAST(web_page.wp_web_page_sk AS Float64)] + │ FilterExec: wp_char_count@1 >= 5000 AND wp_char_count@1 <= 5200, projection=[wp_web_page_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_page/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-3.parquet]]}, projection=[wp_web_page_sk, wp_char_count], file_type=parquet, predicate=wp_char_count@10 >= 5000 AND wp_char_count@10 <= 5200, pruning_predicate=wp_char_count_null_count@1 != row_count@2 AND wp_char_count_max@0 >= 5000 AND wp_char_count_null_count@1 != row_count@2 AND wp_char_count_min@3 <= 5200, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] - │ FilterExec: t_hour@1 >= 8 AND t_hour@1 <= 9, projection=[t_time_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour], file_type=parquet, predicate=t_hour@3 >= 8 AND t_hour@3 <= 9, pruning_predicate=t_hour_null_count@1 != row_count@2 AND t_hour_max@0 >= 8 AND t_hour_null_count@1 != row_count@2 AND t_hour_min@3 <= 9, required_guarantees=[] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] + │ FilterExec: t_hour@1 >= 8 AND t_hour@1 <= 9, projection=[t_time_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour], file_type=parquet, predicate=t_hour@3 >= 8 AND t_hour@3 <= 9, pruning_predicate=t_hour_null_count@1 != row_count@2 AND t_hour_max@0 >= 8 AND t_hour_null_count@1 != row_count@2 AND t_hour_min@3 <= 9, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] - │ FilterExec: hd_dep_count@1 = 6, projection=[hd_demo_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] + │ FilterExec: hd_dep_count@1 = 6, projection=[hd_demo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 6] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ ProjectionExec: expr=[wp_web_page_sk@0 as wp_web_page_sk, CAST(wp_web_page_sk@0 AS Float64) as CAST(web_page.wp_web_page_sk AS Float64)] - │ FilterExec: wp_char_count@1 >= 5000 AND wp_char_count@1 <= 5200, projection=[wp_web_page_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_page/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-3.parquet]]}, projection=[wp_web_page_sk, wp_char_count], file_type=parquet, predicate=wp_char_count@10 >= 5000 AND wp_char_count@10 <= 5200, pruning_predicate=wp_char_count_null_count@1 != row_count@2 AND wp_char_count_max@0 >= 5000 AND wp_char_count_null_count@1 != row_count@2 AND wp_char_count_min@3 <= 5200, required_guarantees=[] + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[wp_web_page_sk@0 as wp_web_page_sk, CAST(wp_web_page_sk@0 AS Float64) as CAST(web_page.wp_web_page_sk AS Float64)] + │ FilterExec: wp_char_count@1 >= 5000 AND wp_char_count@1 <= 5200, projection=[wp_web_page_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_page/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_page/part-3.parquet]]}, projection=[wp_web_page_sk, wp_char_count], file_type=parquet, predicate=wp_char_count@10 >= 5000 AND wp_char_count@10 <= 5200, pruning_predicate=wp_char_count_null_count@1 != row_count@2 AND wp_char_count_max@0 >= 5000 AND wp_char_count_null_count@1 != row_count@2 AND wp_char_count_min@3 <= 5200, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] - │ FilterExec: t_hour@1 >= 19 AND t_hour@1 <= 20, projection=[t_time_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour], file_type=parquet, predicate=t_hour@3 >= 19 AND t_hour@3 <= 20, pruning_predicate=t_hour_null_count@1 != row_count@2 AND t_hour_max@0 >= 19 AND t_hour_null_count@1 != row_count@2 AND t_hour_min@3 <= 20, required_guarantees=[] + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] + │ FilterExec: t_hour@1 >= 19 AND t_hour@1 <= 20, projection=[t_time_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour], file_type=parquet, predicate=t_hour@3 >= 19 AND t_hour@3 <= 20, pruning_predicate=t_hour_null_count@1 != row_count@2 AND t_hour_max@0 >= 19 AND t_hour_null_count@1 != row_count@2 AND t_hour_min@3 <= 20, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] - │ FilterExec: hd_dep_count@1 = 6, projection=[hd_demo_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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 7 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] + │ FilterExec: hd_dep_count@1 = 6, projection=[hd_demo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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)] └────────────────────────────────────────────────── "); Ok(()) @@ -10027,66 +10044,66 @@ mod tests { │ 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 │ AggregateExec: mode=Partial, 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@4 as cd_marital_status, cd_education_status@5 as cd_education_status], aggr=[sum(catalog_returns.cr_net_loss)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_hdemo_sk@4, CAST(household_demographics.hd_demo_sk AS Float64)@1)], projection=[cc_call_center_id@0, cc_name@1, cc_manager@2, cr_net_loss@3, cd_marital_status@5, cd_education_status@6] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] │ FilterExec: hd_buy_potential@1 LIKE Unknown%, projection=[hd_demo_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/household_demographics/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/household_demographics/part-3.parquet]]}, projection=[hd_demo_sk, hd_buy_potential], file_type=parquet, predicate=hd_buy_potential@2 LIKE Unknown%, pruning_predicate=hd_buy_potential_null_count@2 != row_count@3 AND hd_buy_potential_min@0 <= Unknowo AND Unknown <= hd_buy_potential_max@1, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_cdemo_sk@4, CAST(customer_demographics.cd_demo_sk AS Float64)@3)], projection=[cc_call_center_id@0, cc_name@1, cc_manager@2, cr_net_loss@3, c_current_hdemo_sk@5, cd_marital_status@7, cd_education_status@8] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] - │ FilterExec: cd_marital_status@1 = M AND cd_education_status@2 = Unknown OR cd_marital_status@1 = W AND cd_education_status@2 = Advanced Degree - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_marital_status@2 = M AND cd_education_status@3 = Unknown OR cd_marital_status@2 = W AND cd_education_status@3 = Advanced Degree, pruning_predicate=cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= M AND M <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= Unknown AND Unknown <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= W AND W <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= Advanced Degree AND Advanced Degree <= cd_education_status_max@5, required_guarantees=[cd_education_status in (Advanced Degree, Unknown), cd_marital_status in (M, W)] - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@6, ca_address_sk@0)], projection=[cc_call_center_id@0, cc_name@1, cc_manager@2, cr_net_loss@3, c_current_cdemo_sk@4, c_current_hdemo_sk@5] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ FilterExec: ca_gmt_offset@1 = Some(-700),4,2, projection=[ca_address_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/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_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-700),4,2 AND DynamicFilter [ empty ], pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-700),4,2 AND Some(-700),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-700),4,2)] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cr_returning_customer_sk@3, CAST(customer.c_customer_sk AS Float64)@4)], projection=[cc_call_center_id@0, cc_name@1, cc_manager@2, cr_net_loss@4, c_current_cdemo_sk@6, c_current_hdemo_sk@7, c_current_addr_sk@8] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=3 - │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_cdemo_sk@1 as c_current_cdemo_sk, c_current_hdemo_sk@2 as c_current_hdemo_sk, c_current_addr_sk@3 as c_current_addr_sk, 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,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_hdemo_sk, c_current_addr_sk], file_type=parquet + ┌───── Stage 6 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_cdemo_sk@4, CAST(customer_demographics.cd_demo_sk AS Float64)@3)], projection=[cc_call_center_id@0, cc_name@1, cc_manager@2, cr_net_loss@3, c_current_hdemo_sk@5, cd_marital_status@7, cd_education_status@8] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[cd_demo_sk@0 as cd_demo_sk, cd_marital_status@1 as cd_marital_status, cd_education_status@2 as cd_education_status, CAST(cd_demo_sk@0 AS Float64) as CAST(customer_demographics.cd_demo_sk AS Float64)] + │ FilterExec: cd_marital_status@1 = M AND cd_education_status@2 = Unknown OR cd_marital_status@1 = W AND cd_education_status@2 = Advanced Degree + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/customer_demographics/part-2.parquet:..], ...]}, projection=[cd_demo_sk, cd_marital_status, cd_education_status], file_type=parquet, predicate=cd_marital_status@2 = M AND cd_education_status@3 = Unknown OR cd_marital_status@2 = W AND cd_education_status@3 = Advanced Degree, pruning_predicate=cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= M AND M <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= Unknown AND Unknown <= cd_education_status_max@5 OR cd_marital_status_null_count@2 != row_count@3 AND cd_marital_status_min@0 <= W AND W <= cd_marital_status_max@1 AND cd_education_status_null_count@6 != row_count@3 AND cd_education_status_min@4 <= Advanced Degree AND Advanced Degree <= cd_education_status_max@5, required_guarantees=[cd_education_status in (Advanced Degree, Unknown), cd_marital_status in (M, W)] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_current_addr_sk@6, ca_address_sk@0)], projection=[cc_call_center_id@0, cc_name@1, cc_manager@2, cr_net_loss@3, c_current_cdemo_sk@4, c_current_hdemo_sk@5] + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ FilterExec: ca_gmt_offset@1 = Some(-700),4,2, projection=[ca_address_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/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_gmt_offset], file_type=parquet, predicate=ca_gmt_offset@11 = Some(-700),4,2 AND DynamicFilter [ empty ], pruning_predicate=ca_gmt_offset_null_count@2 != row_count@3 AND ca_gmt_offset_min@0 <= Some(-700),4,2 AND Some(-700),4,2 <= ca_gmt_offset_max@1, required_guarantees=[ca_gmt_offset in (Some(-700),4,2)] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cr_returning_customer_sk@3, CAST(customer.c_customer_sk AS Float64)@4)], projection=[cc_call_center_id@0, cc_name@1, cc_manager@2, cr_net_loss@4, c_current_cdemo_sk@6, c_current_hdemo_sk@7, c_current_addr_sk@8] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ ProjectionExec: expr=[c_customer_sk@0 as c_customer_sk, c_current_cdemo_sk@1 as c_current_cdemo_sk, c_current_hdemo_sk@2 as c_current_hdemo_sk, c_current_addr_sk@3 as c_current_addr_sk, 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,__] t2:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/customer/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/customer/part-3.parquet]]}, projection=[c_customer_sk, c_current_cdemo_sk, c_current_hdemo_sk, c_current_addr_sk], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] t2:[p18..p26] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cr_returned_date_sk@3, d_date_sk@0)], projection=[cc_call_center_id@0, cc_name@1, cc_manager@2, cr_returning_customer_sk@4, cr_net_loss@5] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ FilterExec: d_year@1 = 1998 AND d_moy@2 = 11, 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, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 11 AND DynamicFilter [ empty ], 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 <= 11 AND 11 <= d_moy_max@5, required_guarantees=[d_moy in (11), d_year in (1998)] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cr_returned_date_sk@3, d_date_sk@0)], projection=[cc_call_center_id@0, cc_name@1, cc_manager@2, cr_returning_customer_sk@4, cr_net_loss@5] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ FilterExec: d_year@1 = 1998 AND d_moy@2 = 11, 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, d_moy], file_type=parquet, predicate=d_year@6 = 1998 AND d_moy@8 = 11 AND DynamicFilter [ empty ], 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 <= 11 AND 11 <= d_moy_max@5, required_guarantees=[d_moy in (11), d_year in (1998)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(call_center.cc_call_center_sk AS Float64)@4, cr_call_center_sk@2)], projection=[cc_call_center_id@1, cc_name@2, cc_manager@3, cr_returned_date_sk@5, cr_returning_customer_sk@6, cr_net_loss@8] - │ CoalescePartitionsExec - │ [Stage 1] => 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_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_returned_date_sk, cr_returning_customer_sk, cr_call_center_sk, cr_net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(call_center.cc_call_center_sk AS Float64)@4, cr_call_center_sk@2)], projection=[cc_call_center_id@1, cc_name@2, cc_manager@3, cr_returned_date_sk@5, cr_returning_customer_sk@6, cr_net_loss@8] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-0.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-1.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], [/testdata/tpcds/plans_sf1_partitions4/catalog_returns/part-2.parquet:..], ...]}, projection=[cr_returned_date_sk, cr_returning_customer_sk, cr_call_center_sk, cr_net_loss], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/call_center/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-3.parquet]]}, projection=[cc_call_center_sk, cc_call_center_id, cc_name, cc_manager, CAST(cc_call_center_sk@0 AS Float64) as CAST(call_center.cc_call_center_sk AS Float64)], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/call_center/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-3.parquet]]}, projection=[cc_call_center_sk, cc_call_center_id, cc_name, cc_manager, CAST(cc_call_center_sk@0 AS Float64) as CAST(call_center.cc_call_center_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── "); Ok(()) @@ -10105,55 +10122,55 @@ mod tests { ┌───── 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 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ ProjectionExec: expr=[CAST(1.3 * CAST(avg(web_sales.ws_ext_discount_amt)@1 AS Float64) AS Decimal128(30, 15)) as Float64(1.3) * avg(web_sales.ws_ext_discount_amt), ws_item_sk@0 as ws_item_sk] - │ AggregateExec: mode=FinalPartitioned, gby=[ws_item_sk@0 as ws_item_sk], aggr=[avg(web_sales.ws_ext_discount_amt)] - │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[CAST(1.3 * CAST(avg(web_sales.ws_ext_discount_amt)@1 AS Float64) AS Decimal128(30, 15)) as Float64(1.3) * avg(web_sales.ws_ext_discount_amt), ws_item_sk@0 as ws_item_sk] + │ AggregateExec: mode=FinalPartitioned, gby=[ws_item_sk@0 as ws_item_sk], aggr=[avg(web_sales.ws_ext_discount_amt)] + │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([ws_item_sk@0], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[ws_item_sk@0 as ws_item_sk], aggr=[avg(web_sales.ws_ext_discount_amt)] │ 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_ext_discount_amt@4] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_date@1 >= 2000-01-27 AND d_date@1 <= 2000-04-26, projection=[d_date_sk@0] - │ 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, predicate=d_date@2 >= 2000-01-27 AND d_date@2 <= 2000-04-26, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-01-27 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-26, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_date@1 >= 2000-01-27 AND d_date@1 <= 2000-04-26, projection=[d_date_sk@0] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_date@1 >= 2000-01-27 AND d_date@1 <= 2000-04-26, projection=[d_date_sk@0] + │ 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, predicate=d_date@2 >= 2000-01-27 AND d_date@2 <= 2000-04-26, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-01-27 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-26, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_date@1 >= 2000-01-27 AND d_date@1 <= 2000-04-26, projection=[d_date_sk@0] + │ 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, predicate=d_date@2 >= 2000-01-27 AND d_date@2 <= 2000-04-26, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-01-27 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-26, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: i_manufact_id@1 = 350, projection=[i_item_sk@0] │ 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, predicate=d_date@2 >= 2000-01-27 AND d_date@2 <= 2000-04-26, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 2000-01-27 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 2000-04-26, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_manufact_id@1 = 350, projection=[i_item_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_manufact_id], file_type=parquet, predicate=i_manufact_id@13 = 350 AND DynamicFilter [ empty ], pruning_predicate=i_manufact_id_null_count@2 != row_count@3 AND i_manufact_id_min@0 <= 350 AND 350 <= i_manufact_id_max@1, required_guarantees=[i_manufact_id in (350)] + │ 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=i_manufact_id@13 = 350 AND DynamicFilter [ empty ], pruning_predicate=i_manufact_id_null_count@2 != row_count@3 AND i_manufact_id_min@0 <= 350 AND 350 <= i_manufact_id_max@1, required_guarantees=[i_manufact_id in (350)] └────────────────────────────────────────────────── "); Ok(()) @@ -10177,20 +10194,20 @@ mod tests { │ AggregateExec: mode=Partial, gby=[ss_customer_sk@0 as ss_customer_sk], aggr=[sum(t.act_sales)] │ ProjectionExec: expr=[ss_customer_sk@0 as ss_customer_sk, CASE WHEN sr_return_quantity@3 IS NOT NULL THEN (ss_quantity@1 - sr_return_quantity@3) * CAST(ss_sales_price@2 AS Float64) ELSE ss_quantity@1 * CAST(ss_sales_price@2 AS Float64) END as act_sales] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(reason.r_reason_sk AS Float64)@1, sr_reason_sk@3)], projection=[ss_customer_sk@2, ss_quantity@3, ss_sales_price@4, sr_return_quantity@6] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[ss_customer_sk@2 as ss_customer_sk, ss_quantity@3 as ss_quantity, ss_sales_price@4 as ss_sales_price, sr_reason_sk@0 as sr_reason_sk, sr_return_quantity@1 as sr_return_quantity] │ HashJoinExec: mode=Partitioned, join_type=Right, on=[(sr_item_sk@0, ss_item_sk@0), (sr_ticket_number@2, ss_ticket_number@2)], projection=[sr_reason_sk@1, sr_return_quantity@3, ss_customer_sk@5, ss_quantity@7, ss_sales_price@8] │ [Stage 2] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ [Stage 3] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[r_reason_sk@0 as r_reason_sk, CAST(r_reason_sk@0 AS Float64) as CAST(reason.r_reason_sk AS Float64)] - │ FilterExec: r_reason_desc@1 = reason 28, projection=[r_reason_sk@0] - │ 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, r_reason_desc], file_type=parquet, predicate=r_reason_desc@2 = reason 28, pruning_predicate=r_reason_desc_null_count@2 != row_count@3 AND r_reason_desc_min@0 <= reason 28 AND reason 28 <= r_reason_desc_max@1, required_guarantees=[r_reason_desc in (reason 28)] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[r_reason_sk@0 as r_reason_sk, CAST(r_reason_sk@0 AS Float64) as CAST(reason.r_reason_sk AS Float64)] + │ FilterExec: r_reason_desc@1 = reason 28, projection=[r_reason_sk@0] + │ 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, r_reason_desc], file_type=parquet, predicate=r_reason_desc@2 = reason 28, pruning_predicate=r_reason_desc_null_count@2 != row_count@3 AND r_reason_desc_min@0 <= reason 28 AND reason 28 <= r_reason_desc_max@1, required_guarantees=[r_reason_desc in (reason 28)] └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p0..p8] t2:[p0..p8] │ RepartitionExec: partitioning=Hash([sr_item_sk@0, sr_ticket_number@2], 9), input_partitions=2 @@ -10229,40 +10246,40 @@ mod tests { └────────────────────────────────────────────────── ┌───── 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 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ ProjectionExec: expr=[web_site_sk@0 as web_site_sk, CAST(web_site_sk@0 AS Float64) as CAST(web_site.web_site_sk AS Float64)] - │ FilterExec: web_company_name@1 = pri, projection=[web_site_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_site/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-3.parquet]]}, projection=[web_site_sk, web_company_name], file_type=parquet, predicate=web_company_name@14 = pri, pruning_predicate=web_company_name_null_count@2 != row_count@3 AND web_company_name_min@0 <= pri AND pri <= web_company_name_max@1, required_guarantees=[web_company_name in (pri)] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[web_site_sk@0 as web_site_sk, CAST(web_site_sk@0 AS Float64) as CAST(web_site.web_site_sk AS Float64)] + │ FilterExec: web_company_name@1 = pri, projection=[web_site_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_site/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-3.parquet]]}, projection=[web_site_sk, web_company_name], file_type=parquet, predicate=web_company_name@14 = pri, pruning_predicate=web_company_name_null_count@2 != row_count@3 AND web_company_name_min@0 <= pri AND pri <= web_company_name_max@1, required_guarantees=[web_company_name in (pri)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ FilterExec: ca_state@1 = IL, projection=[ca_address_sk@0] - │ 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_state], file_type=parquet, predicate=ca_state@8 = IL, pruning_predicate=ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IL AND IL <= ca_state_max@1, required_guarantees=[ca_state in (IL)] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] + │ FilterExec: ca_state@1 = IL, projection=[ca_address_sk@0] + │ 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_state], file_type=parquet, predicate=ca_state@8 = IL, pruning_predicate=ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IL AND IL <= ca_state_max@1, required_guarantees=[ca_state in (IL)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_date@1 >= 1999-02-01 AND d_date@1 <= 1999-04-02, projection=[d_date_sk@0] - │ 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, predicate=d_date@2 >= 1999-02-01 AND d_date@2 <= 1999-04-02, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 1999-02-01 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 1999-04-02, required_guarantees=[] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_date@1 >= 1999-02-01 AND d_date@1 <= 1999-04-02, projection=[d_date_sk@0] + │ 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, predicate=d_date@2 >= 1999-02-01 AND d_date@2 <= 1999-04-02, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 1999-02-01 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 1999-04-02, required_guarantees=[] └────────────────────────────────────────────────── "); Ok(()) @@ -10289,8 +10306,8 @@ mod tests { │ [Stage 5] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ [Stage 6] => NetworkShuffleExec: output_partitions=3, input_tasks=3 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(wr_order_number@0, ws_order_number@0)], projection=[wr_order_number@0] - │ CoalescePartitionsExec - │ BroadcastExec: input_partitions=2, consumer_tasks=1, output_partitions=2 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec │ 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 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ws_order_number@1, ws_order_number@1)], filter=ws_warehouse_sk@1 != ws_warehouse_sk@0, projection=[ws_order_number@1] │ [Stage 7] => NetworkShuffleExec: output_partitions=3, input_tasks=3 @@ -10298,40 +10315,40 @@ mod tests { └────────────────────────────────────────────────── ┌───── 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 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ ProjectionExec: expr=[web_site_sk@0 as web_site_sk, CAST(web_site_sk@0 AS Float64) as CAST(web_site.web_site_sk AS Float64)] - │ FilterExec: web_company_name@1 = pri, projection=[web_site_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_site/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-3.parquet]]}, projection=[web_site_sk, web_company_name], file_type=parquet, predicate=web_company_name@14 = pri, pruning_predicate=web_company_name_null_count@2 != row_count@3 AND web_company_name_min@0 <= pri AND pri <= web_company_name_max@1, required_guarantees=[web_company_name in (pri)] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[web_site_sk@0 as web_site_sk, CAST(web_site_sk@0 AS Float64) as CAST(web_site.web_site_sk AS Float64)] + │ FilterExec: web_company_name@1 = pri, projection=[web_site_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/web_site/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/web_site/part-3.parquet]]}, projection=[web_site_sk, web_company_name], file_type=parquet, predicate=web_company_name@14 = pri, pruning_predicate=web_company_name_null_count@2 != row_count@3 AND web_company_name_min@0 <= pri AND pri <= web_company_name_max@1, required_guarantees=[web_company_name in (pri)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] - │ FilterExec: ca_state@1 = IL, projection=[ca_address_sk@0] - │ 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_state], file_type=parquet, predicate=ca_state@8 = IL, pruning_predicate=ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IL AND IL <= ca_state_max@1, required_guarantees=[ca_state in (IL)] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[ca_address_sk@0 as ca_address_sk, CAST(ca_address_sk@0 AS Float64) as CAST(customer_address.ca_address_sk AS Float64)] + │ FilterExec: ca_state@1 = IL, projection=[ca_address_sk@0] + │ 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_state], file_type=parquet, predicate=ca_state@8 = IL, pruning_predicate=ca_state_null_count@2 != row_count@3 AND ca_state_min@0 <= IL AND IL <= ca_state_max@1, required_guarantees=[ca_state in (IL)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_date@1 >= 1999-02-01 AND d_date@1 <= 1999-04-02, projection=[d_date_sk@0] - │ 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, predicate=d_date@2 >= 1999-02-01 AND d_date@2 <= 1999-04-02, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 1999-02-01 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 1999-04-02, required_guarantees=[] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_date@1 >= 1999-02-01 AND d_date@1 <= 1999-04-02, projection=[d_date_sk@0] + │ 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, predicate=d_date@2 >= 1999-02-01 AND d_date@2 <= 1999-04-02, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 1999-02-01 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 1999-04-02, required_guarantees=[] └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ RepartitionExec: partitioning=Hash([ws_order_number@1], 3), input_partitions=2 @@ -10371,40 +10388,40 @@ mod tests { │ 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 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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 - │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] - │ FilterExec: s_store_name@1 = ese, projection=[s_store_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[s_store_sk@0 as s_store_sk, CAST(s_store_sk@0 AS Float64) as CAST(store.s_store_sk AS Float64)] + │ FilterExec: s_store_name@1 = ese, projection=[s_store_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/store/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/store/part-3.parquet]]}, projection=[s_store_sk, s_store_name], file_type=parquet, predicate=s_store_name@5 = ese, pruning_predicate=s_store_name_null_count@2 != row_count@3 AND s_store_name_min@0 <= ese AND ese <= s_store_name_max@1, required_guarantees=[s_store_name in (ese)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] - │ FilterExec: t_hour@1 = 20 AND t_minute@2 >= 30, projection=[t_time_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute], file_type=parquet, predicate=t_hour@3 = 20 AND t_minute@4 >= 30, pruning_predicate=t_hour_null_count@2 != row_count@3 AND t_hour_min@0 <= 20 AND 20 <= t_hour_max@1 AND t_minute_null_count@5 != row_count@3 AND t_minute_max@4 >= 30, required_guarantees=[t_hour in (20)] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[t_time_sk@0 as t_time_sk, CAST(t_time_sk@0 AS Float64) as CAST(time_dim.t_time_sk AS Float64)] + │ FilterExec: t_hour@1 = 20 AND t_minute@2 >= 30, projection=[t_time_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/time_dim/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/time_dim/part-3.parquet]]}, projection=[t_time_sk, t_hour, t_minute], file_type=parquet, predicate=t_hour@3 = 20 AND t_minute@4 >= 30, pruning_predicate=t_hour_null_count@2 != row_count@3 AND t_hour_min@0 <= 20 AND 20 <= t_hour_max@1 AND t_minute_null_count@5 != row_count@3 AND t_minute_max@4 >= 30, required_guarantees=[t_hour in (20)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] - │ FilterExec: hd_dep_count@1 = 7, projection=[hd_demo_sk@0] - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ 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 = 7, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 7 AND 7 <= hd_dep_count_max@1, required_guarantees=[hd_dep_count in (7)] + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[hd_demo_sk@0 as hd_demo_sk, CAST(hd_demo_sk@0 AS Float64) as CAST(household_demographics.hd_demo_sk AS Float64)] + │ FilterExec: hd_dep_count@1 = 7, projection=[hd_demo_sk@0] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 + │ 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 = 7, pruning_predicate=hd_dep_count_null_count@2 != row_count@3 AND hd_dep_count_min@0 <= 7 AND 7 <= hd_dep_count_max@1, required_guarantees=[hd_dep_count in (7)] └────────────────────────────────────────────────── "); Ok(()) @@ -10436,35 +10453,35 @@ mod tests { │ RepartitionExec: partitioning=Hash([cs_bill_customer_sk@0, cs_item_sk@1], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[cs_bill_customer_sk@0 as cs_bill_customer_sk, cs_item_sk@1 as cs_item_sk], aggr=[] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, cs_sold_date_sk@0)], projection=[cs_bill_customer_sk@3, cs_item_sk@4] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_bill_customer_sk, cs_item_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_month_seq@1 >= 1200 AND d_month_seq@1 <= 1211, projection=[d_date_sk@0] - │ 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_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_month_seq@1 >= 1200 AND d_month_seq@1 <= 1211, projection=[d_date_sk@0] + │ 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_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] │ RepartitionExec: partitioning=Hash([ss_customer_sk@0, ss_item_sk@1], 3), input_partitions=2 │ AggregateExec: mode=Partial, gby=[ss_customer_sk@1 as ss_customer_sk, ss_item_sk@0 as ss_item_sk], aggr=[] │ 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_customer_sk@4] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_customer_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ 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_month_seq@1 >= 1200 AND d_month_seq@1 <= 1211, projection=[d_date_sk@0] - │ 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_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_month_seq@1 >= 1200 AND d_month_seq@1 <= 1211, projection=[d_date_sk@0] + │ 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_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── "); Ok(()) @@ -10490,29 +10507,29 @@ mod tests { │ RepartitionExec: partitioning=Hash([i_item_id@0, i_item_desc@1, i_category@2, i_class@3, i_current_price@4], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[i_item_id@1 as i_item_id, i_item_desc@2 as i_item_desc, i_category@5 as i_category, i_class@4 as i_class, i_current_price@3 as i_current_price], aggr=[sum(store_sales.ss_ext_sales_price)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(date_dim.d_date_sk AS Float64)@1, ss_sold_date_sk@0)], projection=[ss_ext_sales_price@3, i_item_id@4, i_item_desc@5, i_current_price@6, i_class@7, i_category@8] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=2 │ ProjectionExec: expr=[ss_sold_date_sk@5 as ss_sold_date_sk, ss_ext_sales_price@6 as ss_ext_sales_price, i_item_id@0 as i_item_id, i_item_desc@1 as i_item_desc, i_current_price@2 as i_current_price, i_class@3 as i_class, i_category@4 as i_category] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(i_item_sk@0, ss_item_sk@1)], projection=[i_item_id@1, i_item_desc@2, i_current_price@3, i_class@4, i_category@5, ss_sold_date_sk@6, ss_ext_sales_price@8] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=3, stage_partitions=9, input_tasks=2 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_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 - │ 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_date@1 >= 1999-02-22 AND d_date@1 <= 1999-03-24, projection=[d_date_sk@0] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ 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_date@1 >= 1999-02-22 AND d_date@1 <= 1999-03-24, projection=[d_date_sk@0] + │ 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, predicate=d_date@2 >= 1999-02-22 AND d_date@2 <= 1999-03-24, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 1999-02-22 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 1999-03-24, required_guarantees=[] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ FilterExec: i_category@5 = Sports OR i_category@5 = Books OR i_category@5 = Home │ 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, predicate=d_date@2 >= 1999-02-22 AND d_date@2 <= 1999-03-24, pruning_predicate=d_date_null_count@1 != row_count@2 AND d_date_max@0 >= 1999-02-22 AND d_date_null_count@1 != row_count@2 AND d_date_min@3 <= 1999-03-24, required_guarantees=[] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p8] t1:[p9..p17] - │ BroadcastExec: input_partitions=3, consumer_tasks=3, output_partitions=9 - │ FilterExec: i_category@5 = Sports OR i_category@5 = Books OR i_category@5 = Home - │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/item/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/item/part-3.parquet]]}, projection=[i_item_sk, i_item_id, i_item_desc, i_current_price, i_class, i_category], file_type=parquet, predicate=i_category@12 = Sports OR i_category@12 = Books OR i_category@12 = Home, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Sports AND Sports <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Home AND Home <= i_category_max@1, required_guarantees=[i_category in (Books, Home, Sports)] + │ 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, i_item_desc, i_current_price, i_class, i_category], file_type=parquet, predicate=i_category@12 = Sports OR i_category@12 = Books OR i_category@12 = Home, pruning_predicate=i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Sports AND Sports <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Books AND Books <= i_category_max@1 OR i_category_null_count@2 != row_count@3 AND i_category_min@0 <= Home AND Home <= i_category_max@1, required_guarantees=[i_category in (Books, Home, Sports)] └────────────────────────────────────────────────── "#); Ok(()) @@ -10536,43 +10553,43 @@ mod tests { │ AggregateExec: mode=Partial, gby=[w_substr@1 as w_substr, sm_type@2 as sm_type, cc_name@3 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)] │ ProjectionExec: expr=[cs_ship_date_sk@1 - cs_sold_date_sk@0 as __common_expr_1, w_substr@2 as w_substr, sm_type@3 as sm_type, cc_name@4 as cc_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_ship_date_sk@1, CAST(date_dim.d_date_sk AS Float64)@1)], projection=[cs_sold_date_sk@0, cs_ship_date_sk@1, w_substr@2, sm_type@3, cc_name@4] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_month_seq@1 >= 1200 AND d_month_seq@1 <= 1211, 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_month_seq], file_type=parquet, predicate=d_month_seq@3 >= 1200 AND d_month_seq@3 <= 1211, pruning_predicate=d_month_seq_null_count@1 != row_count@2 AND d_month_seq_max@0 >= 1200 AND d_month_seq_null_count@1 != row_count@2 AND d_month_seq_min@3 <= 1211, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_call_center_sk@2, CAST(call_center.cc_call_center_sk AS Float64)@2)], projection=[cs_sold_date_sk@0, cs_ship_date_sk@1, w_substr@3, sm_type@4, cc_name@6] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/call_center/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-3.parquet]]}, projection=[cc_call_center_sk, cc_name, CAST(cc_call_center_sk@0 AS Float64) as CAST(call_center.cc_call_center_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_ship_mode_sk@3, CAST(ship_mode.sm_ship_mode_sk AS Float64)@2)], projection=[cs_sold_date_sk@0, cs_ship_date_sk@1, cs_call_center_sk@2, w_substr@4, sm_type@6] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=2, stage_partitions=6, input_tasks=3 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-3.parquet]]}, projection=[sm_ship_mode_sk, sm_type, CAST(sm_ship_mode_sk@0 AS Float64) as CAST(ship_mode.sm_ship_mode_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p6..p11] t2:[p12..p17] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_ship_date_sk@2 as cs_ship_date_sk, cs_call_center_sk@3 as cs_call_center_sk, cs_ship_mode_sk@4 as cs_ship_mode_sk, w_substr@0 as w_substr] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(sq1.w_warehouse_sk AS Float64)@2, cs_warehouse_sk@4)], projection=[w_substr@0, cs_sold_date_sk@3, cs_ship_date_sk@4, cs_call_center_sk@5, cs_ship_mode_sk@6] - │ CoalescePartitionsExec - │ [Stage 1] => 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_ship_date_sk, cs_call_center_sk, cs_ship_mode_sk, cs_warehouse_sk], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_call_center_sk@2, CAST(call_center.cc_call_center_sk AS Float64)@2)], projection=[cs_sold_date_sk@0, cs_ship_date_sk@1, w_substr@3, sm_type@4, cc_name@6] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/call_center/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/call_center/part-3.parquet]]}, projection=[cc_call_center_sk, cc_name, CAST(cc_call_center_sk@0 AS Float64) as CAST(call_center.cc_call_center_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(cs_ship_mode_sk@3, CAST(ship_mode.sm_ship_mode_sk AS Float64)@2)], projection=[cs_sold_date_sk@0, cs_ship_date_sk@1, cs_call_center_sk@2, w_substr@4, sm_type@6] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=3 + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,__] t2:[__,__,__,p0] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/ship_mode/part-3.parquet]]}, projection=[sm_ship_mode_sk, sm_type, CAST(sm_ship_mode_sk@0 AS Float64) as CAST(ship_mode.sm_ship_mode_sk AS Float64)], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p3..p5] t2:[p6..p8] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[cs_sold_date_sk@1 as cs_sold_date_sk, cs_ship_date_sk@2 as cs_ship_date_sk, cs_call_center_sk@3 as cs_call_center_sk, cs_ship_mode_sk@4 as cs_ship_mode_sk, w_substr@0 as w_substr] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(sq1.w_warehouse_sk AS Float64)@2, cs_warehouse_sk@4)], projection=[w_substr@0, cs_sold_date_sk@3, cs_ship_date_sk@4, cs_call_center_sk@5, cs_ship_mode_sk@6] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, 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_ship_date_sk, cs_call_center_sk, cs_ship_mode_sk, cs_warehouse_sk], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p6..p11] - │ BroadcastExec: input_partitions=2, consumer_tasks=3, output_partitions=6 - │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] - │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[substr(w_warehouse_name@2, 1, 20) as w_substr, w_warehouse_sk, CAST(w_warehouse_sk@0 AS Float64) as CAST(sq1.w_warehouse_sk AS Float64)], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p3..p5] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ CoalescePartitionsExec + │ PartitionIsolatorExec: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ DataSourceExec: file_groups={4 groups: [[/testdata/tpcds/plans_sf1_partitions4/warehouse/part-0.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-1.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-2.parquet], [/testdata/tpcds/plans_sf1_partitions4/warehouse/part-3.parquet]]}, projection=[substr(w_warehouse_name@2, 1, 20) as w_substr, w_warehouse_sk, CAST(w_warehouse_sk@0 AS Float64) as CAST(sq1.w_warehouse_sk AS Float64)], file_type=parquet └────────────────────────────────────────────────── "); Ok(()) diff --git a/tests/tpch_plans_test.rs b/tests/tpch_plans_test.rs index 2d03fbfb..f271418a 100644 --- a/tests/tpch_plans_test.rs +++ b/tests/tpch_plans_test.rs @@ -58,43 +58,43 @@ mod tests { ┌───── 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 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@9)], projection=[p_partkey@1, p_mfgr@2, s_name@3, s_address@4, s_phone@5, s_acctbal@6, s_comment@7, ps_supplycost@8, n_name@9] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ ProjectionExec: expr=[p_partkey@2 as p_partkey, p_mfgr@3 as p_mfgr, s_name@4 as s_name, s_address@5 as s_address, s_phone@6 as s_phone, s_acctbal@7 as s_acctbal, s_comment@8 as s_comment, ps_supplycost@9 as ps_supplycost, n_name@0 as n_name, n_regionkey@1 as n_regionkey] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@4)], projection=[n_name@1, n_regionkey@2, p_partkey@3, p_mfgr@4, s_name@5, s_address@6, s_phone@8, s_acctbal@9, s_comment@10, ps_supplycost@11] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ ProjectionExec: expr=[p_partkey@6 as p_partkey, p_mfgr@7 as p_mfgr, s_name@0 as s_name, s_address@1 as s_address, s_nationkey@2 as s_nationkey, s_phone@3 as s_phone, s_acctbal@4 as s_acctbal, s_comment@5 as s_comment, ps_supplycost@8 as ps_supplycost] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@2)], projection=[s_name@1, s_address@2, s_nationkey@3, s_phone@4, s_acctbal@5, s_comment@6, p_partkey@7, p_mfgr@8, ps_supplycost@10] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_partkey@0, p_mfgr@1, ps_suppkey@3, ps_supplycost@4] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, 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_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=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 - │ FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] - │ 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/region/1.parquet], [/testdata/tpch/plan_sf0.02/region/10.parquet], [/testdata/tpch/plan_sf0.02/region/11.parquet], [/testdata/tpch/plan_sf0.02/region/12.parquet], [/testdata/tpch/plan_sf0.02/region/13.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = EUROPE, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= EUROPE AND EUROPE <= r_name_max@1, required_guarantees=[r_name in (EUROPE)] - └────────────────────────────────────────────────── - ┌───── 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 - │ 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/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_name, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] + │ 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/region/1.parquet], [/testdata/tpch/plan_sf0.02/region/10.parquet], [/testdata/tpch/plan_sf0.02/region/11.parquet], [/testdata/tpch/plan_sf0.02/region/12.parquet], [/testdata/tpch/plan_sf0.02/region/13.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = EUROPE, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= EUROPE AND EUROPE <= r_name_max@1, required_guarantees=[r_name in (EUROPE)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ 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, s_phone, s_acctbal, s_comment], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ 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/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_name, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@1] + ┌───── Stage 3 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec │ 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/part/1.parquet], [/testdata/tpch/plan_sf0.02/part/10.parquet], [/testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet], [/testdata/tpch/plan_sf0.02/part/13.parquet], ...]}, projection=[p_partkey, p_mfgr, p_type, p_size], file_type=parquet, predicate=p_size@5 = 15 AND p_type@4 LIKE %BRASS, pruning_predicate=p_size_null_count@2 != row_count@3 AND p_size_min@0 <= 15 AND 15 <= p_size_max@1, required_guarantees=[p_size in (15)] + │ 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, s_phone, s_acctbal, s_comment], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@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/part/1.parquet], [/testdata/tpch/plan_sf0.02/part/10.parquet], [/testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet], [/testdata/tpch/plan_sf0.02/part/13.parquet], ...]}, projection=[p_partkey, p_mfgr, p_type, p_size], file_type=parquet, predicate=p_size@5 = 15 AND p_type@4 LIKE %BRASS, pruning_predicate=p_size_null_count@2 != row_count@3 AND p_size_min@0 <= 15 AND 15 <= p_size_max@1, required_guarantees=[p_size in (15)] └────────────────────────────────────────────────── ┌───── Stage 10 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] │ RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 24), input_partitions=6 @@ -106,34 +106,34 @@ mod tests { │ RepartitionExec: partitioning=Hash([ps_partkey@0], 18), input_partitions=4 │ AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@2)], projection=[ps_partkey@1, ps_supplycost@2] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ ProjectionExec: expr=[ps_partkey@1 as ps_partkey, ps_supplycost@2 as ps_supplycost, n_regionkey@0 as n_regionkey] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_regionkey@1, ps_partkey@2, ps_supplycost@3] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ ProjectionExec: expr=[ps_partkey@1 as ps_partkey, ps_supplycost@2 as ps_supplycost, s_nationkey@0 as s_nationkey] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@1)], projection=[s_nationkey@1, ps_partkey@2, ps_supplycost@4] - │ CoalescePartitionsExec - │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 8] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, 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_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] - │ 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/region/1.parquet], [/testdata/tpch/plan_sf0.02/region/10.parquet], [/testdata/tpch/plan_sf0.02/region/11.parquet], [/testdata/tpch/plan_sf0.02/region/12.parquet], [/testdata/tpch/plan_sf0.02/region/13.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = EUROPE, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= EUROPE AND EUROPE <= r_name_max@1, required_guarantees=[r_name in (EUROPE)] + ┌───── Stage 6 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] + │ 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/region/1.parquet], [/testdata/tpch/plan_sf0.02/region/10.parquet], [/testdata/tpch/plan_sf0.02/region/11.parquet], [/testdata/tpch/plan_sf0.02/region/12.parquet], [/testdata/tpch/plan_sf0.02/region/13.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = EUROPE, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= EUROPE AND EUROPE <= r_name_max@1, required_guarantees=[r_name in (EUROPE)] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ 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/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 7 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ 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/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ 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_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 8 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ 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_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── "); Ok(()) @@ -157,26 +157,26 @@ mod tests { │ RepartitionExec: partitioning=Hash([l_orderkey@0, o_orderdate@1, o_shippriority@2], 18), input_partitions=4 │ AggregateExec: mode=Partial, gby=[l_orderkey@2 as l_orderkey, o_orderdate@0 as o_orderdate, o_shippriority@1 as o_shippriority], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, o_shippriority@2, l_orderkey@3, l_extendedprice@4, l_discount@5] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@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_orderkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@10 > 1995-03-15 AND DynamicFilter [ empty ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 > 1995-03-15, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── 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 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ FilterExec: o_orderdate@2 < 1995-03-15 - │ 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_custkey, o_orderdate, o_shippriority], file_type=parquet, predicate=o_orderdate@4 < 1995-03-15 AND DynamicFilter [ empty ], pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@0 < 1995-03-15, required_guarantees=[] + ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 + │ FilterExec: o_orderdate@2 < 1995-03-15 + │ 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_custkey, o_orderdate, o_shippriority], file_type=parquet, predicate=o_orderdate@4 < 1995-03-15 AND DynamicFilter [ empty ], pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@0 < 1995-03-15, 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 - │ FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0] - │ 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_mktsegment], file_type=parquet, predicate=c_mktsegment@6 = BUILDING, pruning_predicate=c_mktsegment_null_count@2 != row_count@3 AND c_mktsegment_min@0 <= BUILDING AND BUILDING <= c_mktsegment_max@1, required_guarantees=[c_mktsegment in (BUILDING)] + ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0] + │ 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_mktsegment], file_type=parquet, predicate=c_mktsegment@6 = BUILDING, pruning_predicate=c_mktsegment_null_count@2 != row_count@3 AND c_mktsegment_min@0 <= BUILDING AND BUILDING <= c_mktsegment_max@1, required_guarantees=[c_mktsegment in (BUILDING)] └────────────────────────────────────────────────── "); Ok(()) @@ -226,51 +226,51 @@ mod tests { │ RepartitionExec: partitioning=Hash([n_name@0], 18), input_partitions=4 │ AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@3)], projection=[l_extendedprice@1, l_discount@2, n_name@3] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, n_name@0 as n_name, n_regionkey@1 as n_regionkey] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_name@1, n_regionkey@2, l_extendedprice@3, l_discount@4] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, s_nationkey@0 as s_nationkey] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1), (s_nationkey@1, c_nationkey@0)], projection=[s_nationkey@1, l_extendedprice@4, l_discount@5] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@1, l_orderkey@0)], projection=[c_nationkey@0, l_suppkey@3, l_extendedprice@4, l_discount@5] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, 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/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_extendedprice, l_discount], 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 - │ FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] - │ 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/region/1.parquet], [/testdata/tpch/plan_sf0.02/region/10.parquet], [/testdata/tpch/plan_sf0.02/region/11.parquet], [/testdata/tpch/plan_sf0.02/region/12.parquet], [/testdata/tpch/plan_sf0.02/region/13.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = ASIA, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= ASIA AND ASIA <= r_name_max@1, required_guarantees=[r_name in (ASIA)] + ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] + │ 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/region/1.parquet], [/testdata/tpch/plan_sf0.02/region/10.parquet], [/testdata/tpch/plan_sf0.02/region/11.parquet], [/testdata/tpch/plan_sf0.02/region/12.parquet], [/testdata/tpch/plan_sf0.02/region/13.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = ASIA, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= ASIA AND ASIA <= r_name_max@1, required_guarantees=[r_name in (ASIA)] └────────────────────────────────────────────────── - ┌───── 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 - │ 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/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_name, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ 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/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_name, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ 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_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 3 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ 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_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_nationkey@1, o_orderkey@2] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@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/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_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@4 >= 1994-01-01 AND o_orderdate@4 < 1995-01-01 AND DynamicFilter [ empty ], pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1994-01-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 < 1995-01-01, required_guarantees=[] + ┌───── Stage 5 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_nationkey@1, o_orderkey@2] + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 + │ FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@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/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_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@4 >= 1994-01-01 AND o_orderdate@4 < 1995-01-01 AND DynamicFilter [ empty ], pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1994-01-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 < 1995-01-01, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ 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_nationkey], file_type=parquet + ┌───── Stage 4 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ 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_nationkey], file_type=parquet └────────────────────────────────────────────────── "); Ok(()) @@ -315,53 +315,53 @@ mod tests { │ AggregateExec: mode=Partial, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)] │ ProjectionExec: expr=[n_name@4 as supp_nation, n_name@0 as cust_nation, date_part(YEAR, l_shipdate@3) as l_year, l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as volume] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, c_nationkey@3)], filter=n_name@0 = FRANCE AND n_name@1 = GERMANY OR n_name@0 = GERMANY AND n_name@1 = FRANCE, projection=[n_name@1, l_extendedprice@2, l_discount@3, l_shipdate@4, n_name@6] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_shipdate@3 as l_shipdate, c_nationkey@4 as c_nationkey, n_name@0 as n_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@0)], projection=[n_name@1, l_extendedprice@3, l_discount@4, l_shipdate@5, c_nationkey@6] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ ProjectionExec: expr=[s_nationkey@1 as s_nationkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, l_shipdate@4 as l_shipdate, c_nationkey@0 as c_nationkey] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@4)], projection=[c_nationkey@1, s_nationkey@2, l_extendedprice@3, l_discount@4, l_shipdate@5] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ ProjectionExec: expr=[s_nationkey@1 as s_nationkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, l_shipdate@4 as l_shipdate, o_custkey@0 as o_custkey] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@1)], projection=[o_custkey@1, s_nationkey@2, l_extendedprice@4, l_discount@5, l_shipdate@6] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5, l_shipdate@6] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 │ 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_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@10 >= 1995-01-01 AND l_shipdate@10 <= 1996-12-31 AND DynamicFilter [ empty ] AND DynamicFilter [ empty ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1995-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 <= 1996-12-31, 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 - │ FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE - │ 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/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = GERMANY OR n_name@1 = FRANCE, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1 OR n_name_null_count@2 != row_count@3 AND n_name_min@0 <= FRANCE AND FRANCE <= n_name_max@1, required_guarantees=[n_name in (FRANCE, GERMANY)] + ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE + │ 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/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = GERMANY OR n_name@1 = FRANCE, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1 OR n_name_null_count@2 != row_count@3 AND n_name_min@0 <= FRANCE AND FRANCE <= n_name_max@1, required_guarantees=[n_name in (FRANCE, GERMANY)] └────────────────────────────────────────────────── - ┌───── 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 - │ FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY - │ 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/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = FRANCE OR n_name@1 = GERMANY, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= FRANCE AND FRANCE <= n_name_max@1 OR n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1, required_guarantees=[n_name in (FRANCE, GERMANY)] + ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY + │ 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/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = FRANCE OR n_name@1 = GERMANY, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= FRANCE AND FRANCE <= n_name_max@1 OR n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1, required_guarantees=[n_name in (FRANCE, GERMANY)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ 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_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 3 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ 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_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ 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_custkey], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ 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_custkey], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ 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_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 5 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ 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_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── "); Ok(()) @@ -386,71 +386,71 @@ mod tests { │ AggregateExec: mode=Partial, gby=[o_year@0 as o_year], aggr=[sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)] │ ProjectionExec: expr=[date_part(YEAR, o_orderdate@2) as o_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume, n_name@3 as nation] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@3)], projection=[l_extendedprice@1, l_discount@2, o_orderdate@3, n_name@5] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, o_orderdate@3 as o_orderdate, n_regionkey@4 as n_regionkey, n_name@0 as n_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_name@1, l_extendedprice@2, l_discount@3, o_orderdate@5, n_regionkey@6] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, s_nationkey@3 as s_nationkey, o_orderdate@4 as o_orderdate, n_regionkey@0 as n_regionkey] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, c_nationkey@4)], projection=[n_regionkey@1, l_extendedprice@2, l_discount@3, s_nationkey@4, o_orderdate@5] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, s_nationkey@3 as s_nationkey, o_orderdate@4 as o_orderdate, c_nationkey@0 as c_nationkey] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@3)], projection=[c_nationkey@1, l_extendedprice@2, l_discount@3, s_nationkey@4, o_orderdate@6] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@4 as s_nationkey, o_custkey@0 as o_custkey, o_orderdate@1 as o_orderdate] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_custkey@1, o_orderdate@2, l_extendedprice@4, l_discount@5, s_nationkey@6] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@0 as s_nationkey] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5] - │ CoalescePartitionsExec - │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_suppkey@3, l_extendedprice@4, l_discount@5] - │ CoalescePartitionsExec - │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 7] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, 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/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_partkey, l_suppkey, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilter [ empty ] 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 - │ FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0] - │ 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/region/1.parquet], [/testdata/tpch/plan_sf0.02/region/10.parquet], [/testdata/tpch/plan_sf0.02/region/11.parquet], [/testdata/tpch/plan_sf0.02/region/12.parquet], [/testdata/tpch/plan_sf0.02/region/13.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = AMERICA, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= AMERICA AND AMERICA <= r_name_max@1, required_guarantees=[r_name in (AMERICA)] - └────────────────────────────────────────────────── - ┌───── 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 - │ 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/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0] + │ 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/region/1.parquet], [/testdata/tpch/plan_sf0.02/region/10.parquet], [/testdata/tpch/plan_sf0.02/region/11.parquet], [/testdata/tpch/plan_sf0.02/region/12.parquet], [/testdata/tpch/plan_sf0.02/region/13.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = AMERICA, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= AMERICA AND AMERICA <= r_name_max@1, required_guarantees=[r_name in (AMERICA)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ 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/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ 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/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ 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_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 3 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ 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/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 + ┌───── Stage 4 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec │ 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_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@4 >= 1995-01-01 AND o_orderdate@4 <= 1996-12-31 AND DynamicFilter [ empty ], pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1995-01-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 <= 1996-12-31, required_guarantees=[] + │ 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_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ 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_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 5 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 + │ 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_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@4 >= 1995-01-01 AND o_orderdate@4 <= 1996-12-31 AND DynamicFilter [ empty ], pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1995-01-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 <= 1996-12-31, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ FilterExec: p_type@1 = ECONOMY ANODIZED STEEL, projection=[p_partkey@0] + ┌───── Stage 6 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec │ 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/part/1.parquet], [/testdata/tpch/plan_sf0.02/part/10.parquet], [/testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet], [/testdata/tpch/plan_sf0.02/part/13.parquet], ...]}, projection=[p_partkey, p_type], file_type=parquet, predicate=p_type@4 = ECONOMY ANODIZED STEEL, pruning_predicate=p_type_null_count@2 != row_count@3 AND p_type_min@0 <= ECONOMY ANODIZED STEEL AND ECONOMY ANODIZED STEEL <= p_type_max@1, required_guarantees=[p_type in (ECONOMY ANODIZED STEEL)] + │ 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_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ FilterExec: p_type@1 = ECONOMY ANODIZED STEEL, projection=[p_partkey@0] + │ 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/part/1.parquet], [/testdata/tpch/plan_sf0.02/part/10.parquet], [/testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet], [/testdata/tpch/plan_sf0.02/part/13.parquet], ...]}, projection=[p_partkey, p_type], file_type=parquet, predicate=p_type@4 = ECONOMY ANODIZED STEEL, pruning_predicate=p_type_null_count@2 != row_count@3 AND p_type_min@0 <= ECONOMY ANODIZED STEEL AND ECONOMY ANODIZED STEEL <= p_type_max@1, required_guarantees=[p_type in (ECONOMY ANODIZED STEEL)] └────────────────────────────────────────────────── "#); Ok(()) @@ -475,51 +475,51 @@ mod tests { │ AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] │ ProjectionExec: expr=[n_name@0 as nation, date_part(YEAR, o_orderdate@5) as o_year, l_extendedprice@2 * (Some(1),20,0 - l_discount@3) - ps_supplycost@4 * l_quantity@1 as amount] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[n_name@1, l_quantity@2, l_extendedprice@3, l_discount@4, ps_supplycost@6, o_orderdate@7] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@4 as s_nationkey, ps_supplycost@5 as ps_supplycost, o_orderdate@0 as o_orderdate] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@7] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_quantity@2 as l_quantity, l_extendedprice@3 as l_extendedprice, l_discount@4 as l_discount, s_nationkey@5 as s_nationkey, ps_supplycost@0 as ps_supplycost] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ps_suppkey@1, l_suppkey@2), (ps_partkey@0, l_partkey@1)], projection=[ps_supplycost@2, l_orderkey@3, l_quantity@6, l_extendedprice@7, l_discount@8, s_nationkey@9] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_partkey@2 as l_partkey, l_suppkey@3 as l_suppkey, l_quantity@4 as l_quantity, l_extendedprice@5 as l_extendedprice, l_discount@6 as l_discount, s_nationkey@0 as s_nationkey] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@2)], projection=[s_nationkey@1, l_orderkey@2, l_partkey@3, l_suppkey@4, l_quantity@5, l_extendedprice@6, l_discount@7] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_partkey@2, l_suppkey@3, l_quantity@4, l_extendedprice@5, l_discount@6] - │ CoalescePartitionsExec - │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, 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/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_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] 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 - │ 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/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet - └────────────────────────────────────────────────── - ┌───── 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 - │ 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], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ 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/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ 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_partkey, ps_suppkey, ps_supplycost], file_type=parquet + ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ 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], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ 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_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 3 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ 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_partkey, ps_suppkey, ps_supplycost], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ FilterExec: p_name@1 LIKE %green%, projection=[p_partkey@0] + ┌───── Stage 4 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec │ 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/part/1.parquet], [/testdata/tpch/plan_sf0.02/part/10.parquet], [/testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet], [/testdata/tpch/plan_sf0.02/part/13.parquet], ...]}, projection=[p_partkey, p_name], file_type=parquet, predicate=p_name@1 LIKE %green% + │ 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_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ FilterExec: p_name@1 LIKE %green%, projection=[p_partkey@0] + │ 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/part/1.parquet], [/testdata/tpch/plan_sf0.02/part/10.parquet], [/testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet], [/testdata/tpch/plan_sf0.02/part/13.parquet], ...]}, projection=[p_partkey, p_name], file_type=parquet, predicate=p_name@1 LIKE %green% └────────────────────────────────────────────────── "); Ok(()) @@ -544,33 +544,33 @@ mod tests { │ AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@4 as c_acctbal, c_phone@3 as c_phone, n_name@8 as n_name, c_address@2 as c_address, c_comment@5 as c_comment], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ ProjectionExec: expr=[c_custkey@1 as c_custkey, c_name@2 as c_name, c_address@3 as c_address, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, l_extendedprice@7 as l_extendedprice, l_discount@8 as l_discount, n_name@0 as n_name] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, c_nationkey@3)], projection=[n_name@1, c_custkey@2, c_name@3, c_address@4, c_phone@6, c_acctbal@7, c_comment@8, l_extendedprice@9, l_discount@10] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@7, l_orderkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@9, l_discount@10] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ FilterExec: l_returnflag@3 = R, projection=[l_orderkey@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_orderkey, l_extendedprice, l_discount, l_returnflag], file_type=parquet, predicate=l_returnflag@8 = R AND DynamicFilter [ empty ], pruning_predicate=l_returnflag_null_count@2 != row_count@3 AND l_returnflag_min@0 <= R AND R <= l_returnflag_max@1, required_guarantees=[l_returnflag in (R)] └────────────────────────────────────────────────── - ┌───── 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 - │ 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/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ 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/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, o_orderkey@7] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@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/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_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@4 >= 1993-10-01 AND o_orderdate@4 < 1994-01-01 AND DynamicFilter [ empty ], pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1993-10-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 < 1994-01-01, required_guarantees=[] + ┌───── Stage 3 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, o_orderkey@7] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 + │ FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@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/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_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@4 >= 1993-10-01 AND o_orderdate@4 < 1994-01-01 AND DynamicFilter [ empty ], pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1993-10-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 < 1994-01-01, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── 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 - │ 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_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ 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_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── "); Ok(()) @@ -582,64 +582,71 @@ mod tests { assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [value@1 DESC] - │ SortExec: expr=[value@1 DESC], preserve_partitioning=[true] - │ ProjectionExec: expr=[ps_partkey@1 as ps_partkey, sum(partsupp.ps_supplycost * partsupp.ps_availqty)@2 as value] - │ NestedLoopJoinExec: join_type=Inner, filter=join_proj_push_down_1@1 > sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)@0, projection=[sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)@0, ps_partkey@1, sum(partsupp.ps_supplycost * partsupp.ps_availqty)@2] - │ 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 - │ 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 8] => NetworkCoalesceExec: output_partitions=18, input_tasks=3 └────────────────────────────────────────────────── - ┌───── 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 8 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] + │ SortExec: expr=[value@1 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[ps_partkey@1 as ps_partkey, sum(partsupp.ps_supplycost * partsupp.ps_availqty)@2 as value] + │ NestedLoopJoinExec: join_type=Inner, filter=join_proj_push_down_1@1 > sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)@0, projection=[sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)@0, ps_partkey@1, sum(partsupp.ps_supplycost * partsupp.ps_availqty)@2] + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=3, input_tasks=1 + │ 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 7] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── - ┌───── 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 - │ FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] - │ 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/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = GERMANY, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1, required_guarantees=[n_name in (GERMANY)] - └────────────────────────────────────────────────── - ┌───── 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 - │ 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_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ RepartitionExec: partitioning=Hash([ps_partkey@0], 6), input_partitions=4 - │ AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[ps_partkey@1, ps_availqty@2, ps_supplycost@3] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 - │ ProjectionExec: expr=[ps_partkey@1 as ps_partkey, ps_availqty@2 as ps_availqty, ps_supplycost@3 as ps_supplycost, s_nationkey@0 as s_nationkey] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@1)], projection=[s_nationkey@1, ps_partkey@2, ps_availqty@4, ps_supplycost@5] - │ CoalescePartitionsExec - │ [Stage 5] => 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_partkey, ps_suppkey, ps_availqty, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ empty ] - └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] - │ 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/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = GERMANY, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1, required_guarantees=[n_name in (GERMANY)] + ┌───── Stage 4 ── Tasks: t0:[p0..p2] + │ BroadcastExec: input_partitions=1, consumer_tasks=3, output_partitions=3 + │ 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 5 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ 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_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── 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] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, 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] + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, 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..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] + │ 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/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = GERMANY, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1, required_guarantees=[n_name in (GERMANY)] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ 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_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] + │ RepartitionExec: partitioning=Hash([ps_partkey@0], 18), input_partitions=4 + │ AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[ps_partkey@1, ps_availqty@2, ps_supplycost@3] + │ [Stage 5] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 + │ ProjectionExec: expr=[ps_partkey@1 as ps_partkey, ps_availqty@2 as ps_availqty, ps_supplycost@3 as ps_supplycost, s_nationkey@0 as s_nationkey] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@1)], projection=[s_nationkey@1, ps_partkey@2, ps_availqty@4, ps_supplycost@5] + │ [Stage 6] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, 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_partkey, ps_suppkey, ps_availqty, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] + │ 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/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = GERMANY, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1, required_guarantees=[n_name in (GERMANY)] + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ 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_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + └────────────────────────────────────────────────── "); Ok(()) } @@ -662,16 +669,16 @@ mod tests { │ RepartitionExec: partitioning=Hash([l_shipmode@0], 18), input_partitions=4 │ AggregateExec: mode=Partial, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, 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/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_orderpriority], 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 - │ FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, projection=[l_orderkey@0, l_shipmode@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/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_shipdate, l_commitdate, l_receiptdate, l_shipmode], file_type=parquet, predicate=(l_shipmode@14 = MAIL OR l_shipmode@14 = SHIP) AND l_receiptdate@12 > l_commitdate@11 AND l_shipdate@10 < l_commitdate@11 AND l_receiptdate@12 >= 1994-01-01 AND l_receiptdate@12 < 1995-01-01, pruning_predicate=(l_shipmode_null_count@2 != row_count@3 AND l_shipmode_min@0 <= MAIL AND MAIL <= l_shipmode_max@1 OR l_shipmode_null_count@2 != row_count@3 AND l_shipmode_min@0 <= SHIP AND SHIP <= l_shipmode_max@1) AND l_receiptdate_null_count@5 != row_count@3 AND l_receiptdate_max@4 >= 1994-01-01 AND l_receiptdate_null_count@5 != row_count@3 AND l_receiptdate_min@6 < 1995-01-01, required_guarantees=[l_shipmode in (MAIL, SHIP)] + ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, projection=[l_orderkey@0, l_shipmode@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/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_shipdate, l_commitdate, l_receiptdate, l_shipmode], file_type=parquet, predicate=(l_shipmode@14 = MAIL OR l_shipmode@14 = SHIP) AND l_receiptdate@12 > l_commitdate@11 AND l_shipdate@10 < l_commitdate@11 AND l_receiptdate@12 >= 1994-01-01 AND l_receiptdate@12 < 1995-01-01, pruning_predicate=(l_shipmode_null_count@2 != row_count@3 AND l_shipmode_min@0 <= MAIL AND MAIL <= l_shipmode_max@1 OR l_shipmode_null_count@2 != row_count@3 AND l_shipmode_min@0 <= SHIP AND SHIP <= l_shipmode_max@1) AND l_receiptdate_null_count@5 != row_count@3 AND l_receiptdate_max@4 >= 1994-01-01 AND l_receiptdate_null_count@5 != row_count@3 AND l_receiptdate_min@6 < 1995-01-01, required_guarantees=[l_shipmode in (MAIL, SHIP)] └────────────────────────────────────────────────── "#); Ok(()) @@ -727,16 +734,16 @@ mod tests { │ 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 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, 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 - │ 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/part/1.parquet], [/testdata/tpch/plan_sf0.02/part/10.parquet], [/testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet], [/testdata/tpch/plan_sf0.02/part/13.parquet], ...]}, projection=[p_partkey, p_type], file_type=parquet + ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ 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/part/1.parquet], [/testdata/tpch/plan_sf0.02/part/10.parquet], [/testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet], [/testdata/tpch/plan_sf0.02/part/13.parquet], ...]}, projection=[p_partkey, p_type], file_type=parquet └────────────────────────────────────────────────── "#); Ok(()) @@ -753,11 +760,9 @@ mod tests { ┌───── 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 + │ [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 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, 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 @@ -781,10 +786,11 @@ mod tests { │ 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_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@10 >= 1996-01-01 AND l_shipdate@10 < 1996-04-01, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1996-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1996-04-01, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ 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_phone], file_type=parquet + ┌───── Stage 4 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ 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_phone], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] │ RepartitionExec: partitioning=Hash([l_suppkey@0], 24), input_partitions=4 @@ -821,26 +827,26 @@ mod tests { │ RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 18), input_partitions=4 │ AggregateExec: mode=Partial, gby=[p_brand@1 as p_brand, p_type@2 as p_type, p_size@3 as p_size, ps_suppkey@0 as alias1], aggr=[] │ HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(s_suppkey@0, ps_suppkey@0)] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=4 │ ProjectionExec: expr=[ps_suppkey@3 as ps_suppkey, p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_brand@1, p_type@2, p_size@3, ps_suppkey@5] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, 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_partkey, ps_suppkey], 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 - │ FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] - │ 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_comment], file_type=parquet, predicate=s_comment@6 LIKE %Customer%Complaints% + ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] + │ 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_comment], file_type=parquet, predicate=s_comment@6 LIKE %Customer%Complaints% └────────────────────────────────────────────────── - ┌───── 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 - │ FilterExec: p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND p_size@3 IN (SET) ([49, 14, 23, 45, 19, 3, 36, 9]) - │ 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/part/1.parquet], [/testdata/tpch/plan_sf0.02/part/10.parquet], [/testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet], [/testdata/tpch/plan_sf0.02/part/13.parquet], ...]}, projection=[p_partkey, p_brand, p_type, p_size], file_type=parquet, predicate=p_brand@3 != Brand#45 AND p_type@4 NOT LIKE MEDIUM POLISHED% AND p_size@5 IN (SET) ([49, 14, 23, 45, 19, 3, 36, 9]), pruning_predicate=p_brand_null_count@2 != row_count@3 AND (p_brand_min@0 != Brand#45 OR Brand#45 != p_brand_max@1) AND p_type_null_count@6 != row_count@3 AND (p_type_min@4 NOT LIKE MEDIUM POLISHED% OR p_type_max@5 NOT LIKE MEDIUM POLISHED%) AND (p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 49 AND 49 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 14 AND 14 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 23 AND 23 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 45 AND 45 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 19 AND 19 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 3 AND 3 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 36 AND 36 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 9 AND 9 <= p_size_max@8), required_guarantees=[p_brand not in (Brand#45), p_size in (14, 19, 23, 3, 36, 45, 49, 9)] + ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ FilterExec: p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND p_size@3 IN (SET) ([49, 14, 23, 45, 19, 3, 36, 9]) + │ 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/part/1.parquet], [/testdata/tpch/plan_sf0.02/part/10.parquet], [/testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet], [/testdata/tpch/plan_sf0.02/part/13.parquet], ...]}, projection=[p_partkey, p_brand, p_type, p_size], file_type=parquet, predicate=p_brand@3 != Brand#45 AND p_type@4 NOT LIKE MEDIUM POLISHED% AND p_size@5 IN (SET) ([49, 14, 23, 45, 19, 3, 36, 9]), pruning_predicate=p_brand_null_count@2 != row_count@3 AND (p_brand_min@0 != Brand#45 OR Brand#45 != p_brand_max@1) AND p_type_null_count@6 != row_count@3 AND (p_type_min@4 NOT LIKE MEDIUM POLISHED% OR p_type_max@5 NOT LIKE MEDIUM POLISHED%) AND (p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 49 AND 49 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 14 AND 14 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 23 AND 23 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 45 AND 45 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 19 AND 19 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 3 AND 3 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 36 AND 36 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 9 AND 9 <= p_size_max@8), required_guarantees=[p_brand not in (Brand#45), p_size in (14, 19, 23, 3, 36, 45, 49, 9)] └────────────────────────────────────────────────── "); Ok(()) @@ -859,26 +865,26 @@ mod tests { ┌───── 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 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, 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 - │ ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, p_partkey@0 as p_partkey] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], projection=[p_partkey@0, l_quantity@2, l_extendedprice@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/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], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, p_partkey@0 as p_partkey] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], projection=[p_partkey@0, l_quantity@2, l_extendedprice@3] + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, 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/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], 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 - │ FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0] - │ 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/part/1.parquet], [/testdata/tpch/plan_sf0.02/part/10.parquet], [/testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet], [/testdata/tpch/plan_sf0.02/part/13.parquet], ...]}, projection=[p_partkey, p_brand, p_container], file_type=parquet, predicate=p_brand@3 = Brand#23 AND p_container@6 = MED BOX, pruning_predicate=p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#23 AND Brand#23 <= p_brand_max@1 AND p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED BOX AND MED BOX <= p_container_max@5, required_guarantees=[p_brand in (Brand#23), p_container in (MED BOX)] + ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0] + │ 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/part/1.parquet], [/testdata/tpch/plan_sf0.02/part/10.parquet], [/testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet], [/testdata/tpch/plan_sf0.02/part/13.parquet], ...]}, projection=[p_partkey, p_brand, p_container], file_type=parquet, predicate=p_brand@3 = Brand#23 AND p_container@6 = MED BOX, pruning_predicate=p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#23 AND Brand#23 <= p_brand_max@1 AND p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED BOX AND MED BOX <= p_container_max@5, required_guarantees=[p_brand in (Brand#23), p_container in (MED BOX)] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] │ RepartitionExec: partitioning=Hash([l_partkey@0], 24), input_partitions=4 @@ -907,19 +913,18 @@ mod tests { │ RepartitionExec: partitioning=Hash([c_name@0, c_custkey@1, o_orderkey@2, o_orderdate@3, o_totalprice@4], 18), input_partitions=4 │ AggregateExec: mode=Partial, gby=[c_name@1 as c_name, c_custkey@0 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@4 as o_orderdate, o_totalprice@3 as o_totalprice], aggr=[sum(lineitem.l_quantity)] │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(l_orderkey@0, o_orderkey@2)] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=6, stage_partitions=24, input_tasks=3 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, input_tasks=3 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@2, l_orderkey@0)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@3, o_orderdate@4, l_quantity@6] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 4] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, 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/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_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0..p23] t1:[p24..p47] t2:[p48..p71] - │ BroadcastExec: input_partitions=6, consumer_tasks=4, output_partitions=24 - │ FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0] - │ AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] - │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0] + │ AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] + │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] │ RepartitionExec: partitioning=Hash([l_orderkey@0], 18), input_partitions=4 @@ -927,18 +932,19 @@ mod tests { │ 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_quantity], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@4, o_orderdate@5] - │ CoalescePartitionsExec - │ [Stage 3] => 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/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_custkey, o_totalprice, o_orderdate], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 4 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@4, o_orderdate@5] + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, 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/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_custkey, o_totalprice, o_orderdate], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ 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_name], file_type=parquet + ┌───── Stage 3 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ 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_name], file_type=parquet └────────────────────────────────────────────────── "); Ok(()) @@ -957,17 +963,17 @@ mod tests { ┌───── 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 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, 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 - │ FilterExec: (p_brand@1 = Brand#12 AND p_container@3 IN (SET) ([SM CASE, SM BOX, SM PACK, SM PKG]) 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 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 p_size@2 <= 15) AND p_size@2 >= 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/part/1.parquet], [/testdata/tpch/plan_sf0.02/part/10.parquet], [/testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet], [/testdata/tpch/plan_sf0.02/part/13.parquet], ...]}, projection=[p_partkey, p_brand, p_size, p_container], file_type=parquet, predicate=(p_brand@3 = Brand#12 AND p_container@6 IN (SET) ([SM CASE, SM BOX, SM PACK, SM PKG]) AND p_size@5 <= 5 OR p_brand@3 = Brand#23 AND p_container@6 IN (SET) ([MED BAG, MED BOX, MED PKG, MED PACK]) AND p_size@5 <= 10 OR p_brand@3 = Brand#34 AND p_container@6 IN (SET) ([LG CASE, LG BOX, LG PACK, LG PKG]) AND p_size@5 <= 15) AND p_size@5 >= 1, pruning_predicate=(p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#12 AND Brand#12 <= p_brand_max@1 AND (p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM CASE AND SM CASE <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM BOX AND SM BOX <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM PACK AND SM PACK <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM PKG AND SM PKG <= p_container_max@5) AND p_size_null_count@8 != row_count@3 AND p_size_min@7 <= 5 OR p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#23 AND Brand#23 <= p_brand_max@1 AND (p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED BAG AND MED BAG <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED BOX AND MED BOX <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED PKG AND MED PKG <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED PACK AND MED PACK <= p_container_max@5) AND p_size_null_count@8 != row_count@3 AND p_size_min@7 <= 10 OR p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#34 AND Brand#34 <= p_brand_max@1 AND (p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG CASE AND LG CASE <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG BOX AND LG BOX <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG PACK AND LG PACK <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG PKG AND LG PKG <= p_container_max@5) AND p_size_null_count@8 != row_count@3 AND p_size_min@7 <= 15) AND p_size_null_count@8 != row_count@3 AND p_size_max@9 >= 1, required_guarantees=[p_brand in (Brand#12, Brand#23, Brand#34), p_container in (LG BOX, LG CASE, LG PACK, LG PKG, MED BAG, MED BOX, MED PACK, MED PKG, SM BOX, SM CASE, SM PACK, SM PKG)] + ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ FilterExec: (p_brand@1 = Brand#12 AND p_container@3 IN (SET) ([SM CASE, SM BOX, SM PACK, SM PKG]) 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 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 p_size@2 <= 15) AND p_size@2 >= 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/part/1.parquet], [/testdata/tpch/plan_sf0.02/part/10.parquet], [/testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet], [/testdata/tpch/plan_sf0.02/part/13.parquet], ...]}, projection=[p_partkey, p_brand, p_size, p_container], file_type=parquet, predicate=(p_brand@3 = Brand#12 AND p_container@6 IN (SET) ([SM CASE, SM BOX, SM PACK, SM PKG]) AND p_size@5 <= 5 OR p_brand@3 = Brand#23 AND p_container@6 IN (SET) ([MED BAG, MED BOX, MED PKG, MED PACK]) AND p_size@5 <= 10 OR p_brand@3 = Brand#34 AND p_container@6 IN (SET) ([LG CASE, LG BOX, LG PACK, LG PKG]) AND p_size@5 <= 15) AND p_size@5 >= 1, pruning_predicate=(p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#12 AND Brand#12 <= p_brand_max@1 AND (p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM CASE AND SM CASE <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM BOX AND SM BOX <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM PACK AND SM PACK <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM PKG AND SM PKG <= p_container_max@5) AND p_size_null_count@8 != row_count@3 AND p_size_min@7 <= 5 OR p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#23 AND Brand#23 <= p_brand_max@1 AND (p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED BAG AND MED BAG <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED BOX AND MED BOX <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED PKG AND MED PKG <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED PACK AND MED PACK <= p_container_max@5) AND p_size_null_count@8 != row_count@3 AND p_size_min@7 <= 10 OR p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#34 AND Brand#34 <= p_brand_max@1 AND (p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG CASE AND LG CASE <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG BOX AND LG BOX <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG PACK AND LG PACK <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG PKG AND LG PKG <= p_container_max@5) AND p_size_null_count@8 != row_count@3 AND p_size_min@7 <= 15) AND p_size_null_count@8 != row_count@3 AND p_size_max@9 >= 1, required_guarantees=[p_brand in (Brand#12, Brand#23, Brand#34), p_container in (LG BOX, LG CASE, LG PACK, LG PKG, MED BAG, MED BOX, MED PACK, MED PKG, SM BOX, SM CASE, SM PACK, SM PKG)] └────────────────────────────────────────────────── "); Ok(()) @@ -984,11 +990,10 @@ mod tests { │ CoalescePartitionsExec │ [Stage 2] => NetworkCoalesceExec: output_partitions=16, 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 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(p_partkey@0, ps_partkey@0)] - │ CoalescePartitionsExec - │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=4, input_tasks=4 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=1, input_tasks=4 │ DataSourceExec: file_groups={6 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, /testdata/tpch/plan_sf0.02/partsupp/14.parquet], [/testdata/tpch/plan_sf0.02/partsupp/15.parquet, /testdata/tpch/plan_sf0.02/partsupp/16.parquet, /testdata/tpch/plan_sf0.02/partsupp/2.parquet], [/testdata/tpch/plan_sf0.02/partsupp/3.parquet, /testdata/tpch/plan_sf0.02/partsupp/4.parquet, /testdata/tpch/plan_sf0.02/partsupp/5.parquet], [/testdata/tpch/plan_sf0.02/partsupp/6.parquet, /testdata/tpch/plan_sf0.02/partsupp/7.parquet, /testdata/tpch/plan_sf0.02/partsupp/8.parquet], ...]}, projection=[ps_partkey, ps_suppkey, ps_availqty], file_type=parquet │ ProjectionExec: expr=[0.5 * CAST(sum(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * sum(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey] │ AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] @@ -996,22 +1001,23 @@ mod tests { └────────────────────────────────────────────────── ┌───── 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 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, 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 - │ FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0] - │ 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/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = CANADA, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= CANADA AND CANADA <= n_name_max@1, required_guarantees=[n_name in (CANADA)] + ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0] + │ 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/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = CANADA, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= CANADA AND CANADA <= n_name_max@1, required_guarantees=[n_name in (CANADA)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] - │ BroadcastExec: input_partitions=4, consumer_tasks=1, output_partitions=4 - │ FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] - │ 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/part/1.parquet], [/testdata/tpch/plan_sf0.02/part/10.parquet], [/testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet], [/testdata/tpch/plan_sf0.02/part/13.parquet], ...]}, projection=[p_partkey, p_name], file_type=parquet, predicate=p_name@1 LIKE forest%, pruning_predicate=p_name_null_count@2 != row_count@3 AND p_name_min@0 <= foresu AND forest <= p_name_max@1, required_guarantees=[] + ┌───── Stage 3 ── Tasks: t0:[p0] t1:[p1] t2:[p2] t3:[p3] + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ CoalescePartitionsExec + │ FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] + │ 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/part/1.parquet], [/testdata/tpch/plan_sf0.02/part/10.parquet], [/testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet], [/testdata/tpch/plan_sf0.02/part/13.parquet], ...]}, projection=[p_partkey, p_name], file_type=parquet, predicate=p_name@1 LIKE forest%, pruning_predicate=p_name_null_count@2 != row_count@3 AND p_name_min@0 <= foresu AND forest <= p_name_max@1, required_guarantees=[] └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 6), input_partitions=4 @@ -1046,34 +1052,34 @@ mod tests { └────────────────────────────────────────────────── ┌───── 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 + │ [Stage 1] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, 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 2] => NetworkBroadcastExec: partitions_per_consumer=4, stage_partitions=16, input_tasks=4 + │ [Stage 2] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, 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 + │ [Stage 3] => NetworkBroadcastExec: partitions_per_consumer=1, stage_partitions=4, 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 - │ FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0] - │ 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/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = SAUDI ARABIA, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= SAUDI ARABIA AND SAUDI ARABIA <= n_name_max@1, required_guarantees=[n_name in (SAUDI ARABIA)] + ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0] + │ 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/nation/1.parquet], [/testdata/tpch/plan_sf0.02/nation/10.parquet], [/testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet], [/testdata/tpch/plan_sf0.02/nation/13.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = SAUDI ARABIA, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= SAUDI ARABIA AND SAUDI ARABIA <= n_name_max@1, required_guarantees=[n_name in (SAUDI ARABIA)] └────────────────────────────────────────────────── - ┌───── 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 - │ FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] - │ 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_orderstatus], file_type=parquet, predicate=o_orderstatus@2 = F, pruning_predicate=o_orderstatus_null_count@2 != row_count@3 AND o_orderstatus_min@0 <= F AND F <= o_orderstatus_max@1, required_guarantees=[o_orderstatus in (F)] + ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] + │ 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_orderstatus], file_type=parquet, predicate=o_orderstatus@2 = F, pruning_predicate=o_orderstatus_null_count@2 != row_count@3 AND o_orderstatus_min@0 <= F AND F <= o_orderstatus_max@1, required_guarantees=[o_orderstatus in (F)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0..p15] t1:[p16..p31] t2:[p32..p47] t3:[p48..p63] - │ BroadcastExec: input_partitions=4, consumer_tasks=4, output_partitions=16 - │ 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_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + ┌───── Stage 3 ── Tasks: t0:[p0..p3] t1:[p4..p7] t2:[p8..p11] t3:[p12..p15] + │ BroadcastExec: input_partitions=1, consumer_tasks=4, output_partitions=4 + │ CoalescePartitionsExec + │ 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_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── "); Ok(()) @@ -1092,9 +1098,10 @@ mod tests { │ AggregateExec: mode=Partial, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)] │ ProjectionExec: expr=[substr(c_phone@1, 1, 2) as cntrycode, c_acctbal@2 as c_acctbal] │ 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 + │ BroadcastExec: input_partitions=1, consumer_tasks=1, output_partitions=1 + │ AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkCoalesceExec: output_partitions=16, 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