diff --git a/benchmarks/src/tpch/run.rs b/benchmarks/src/tpch/run.rs index 618f299..c8a5875 100644 --- a/benchmarks/src/tpch/run.rs +++ b/benchmarks/src/tpch/run.rs @@ -47,7 +47,7 @@ use datafusion_distributed::test_utils::localhost::{ }; use datafusion_distributed::{ DistributedExt, DistributedPhysicalOptimizerRule, DistributedSessionBuilder, - DistributedSessionBuilderContext, StageExec, + DistributedSessionBuilderContext, NetworkBoundaryExt, Stage, }; use log::info; use std::fs; @@ -331,8 +331,8 @@ impl RunOpt { } let mut n_tasks = 0; physical_plan.clone().transform_down(|node| { - if let Some(node) = node.as_any().downcast_ref::() { - n_tasks += node.tasks.len() + if let Some(node) = node.as_network_boundary() { + n_tasks += node.input_stage().map(|v| v.tasks.len()).unwrap_or(0) } Ok(Transformed::no(node)) })?; diff --git a/src/common/mod.rs b/src/common/mod.rs index 9194388..ae81b10 100644 --- a/src/common/mod.rs +++ b/src/common/mod.rs @@ -1,7 +1,4 @@ mod map_last_stream; -mod partitioning; -#[allow(unused)] pub mod ttl_map; pub(crate) use map_last_stream::map_last_stream; -pub(crate) use partitioning::{scale_partitioning, scale_partitioning_props}; diff --git a/src/distributed_physical_optimizer_rule.rs b/src/distributed_physical_optimizer_rule.rs index 1f1829f..a602f6a 100644 --- a/src/distributed_physical_optimizer_rule.rs +++ b/src/distributed_physical_optimizer_rule.rs @@ -1,5 +1,5 @@ -use super::{NetworkShuffleExec, PartitionIsolatorExec, StageExec}; -use crate::execution_plans::NetworkCoalesceExec; +use super::{NetworkShuffleExec, PartitionIsolatorExec, Stage}; +use crate::execution_plans::{DistributedExec, NetworkCoalesceExec}; use datafusion::common::plan_err; use datafusion::common::tree_node::TreeNodeRecursion; use datafusion::datasource::source::DataSourceExec; @@ -99,13 +99,8 @@ impl PhysicalOptimizerRule for DistributedPhysicalOptimizerRule { _config: &ConfigOptions, ) -> Result> { // We can only optimize plans that are not already distributed - if plan.as_any().is::() { - return Ok(plan); - } - let plan = self.apply_network_boundaries(plan)?; - let plan = Self::distribute_plan(plan)?; - Ok(Arc::new(plan)) + Self::distribute_plan(plan) } fn name(&self) -> &str { @@ -199,8 +194,12 @@ impl DistributedPhysicalOptimizerRule { /// This can be used a standalone function for distributing arbitrary plans in which users have /// manually placed network boundaries, or as part of the [DistributedPhysicalOptimizerRule] that /// places the network boundaries automatically as a standard [PhysicalOptimizerRule]. - pub fn distribute_plan(plan: Arc) -> Result { - Self::_distribute_plan_inner(Uuid::new_v4(), plan, &mut 1, 0, 1) + pub fn distribute_plan( + plan: Arc, + ) -> Result, DataFusionError> { + let stage = Self::_distribute_plan_inner(Uuid::new_v4(), plan, &mut 1, 0, 1)?; + let plan = stage.plan.decoded()?; + Ok(Arc::new(DistributedExec::new(Arc::clone(plan)))) } fn _distribute_plan_inner( @@ -209,9 +208,7 @@ impl DistributedPhysicalOptimizerRule { num: &mut usize, depth: usize, n_tasks: usize, - ) -> Result { - let mut inputs = vec![]; - + ) -> Result { let mut distributed = plan.clone().transform_down(|plan| { // We cannot break down CollectLeft hash joins into more than 1 task, as these need // a full materialized build size with all the data in it. @@ -219,7 +216,7 @@ impl DistributedPhysicalOptimizerRule { // Maybe in the future these can be broadcast joins? if let Some(node) = plan.as_any().downcast_ref::() { if n_tasks > 1 && node.mode == PartitionMode::CollectLeft { - return Err(limit_tasks_err(1)) + return Err(limit_tasks_err(1)); } } @@ -232,28 +229,24 @@ impl DistributedPhysicalOptimizerRule { return Ok(Transformed::new(Arc::new(node), true, TreeNodeRecursion::Jump)); } - let mut dnode = if let Some(node) = plan.as_any().downcast_ref::() { - Arc::new(node.clone()) as Arc - } else if let Some(node) = plan.as_any().downcast_ref::() { - Arc::new(node.clone()) as Arc - } else { + let Some(mut dnode) = plan.as_network_boundary().map(Referenced::Borrowed) else { return Ok(Transformed::no(plan)); }; let stage = loop { - let (inner_plan, in_tasks) = dnode.to_stage_info(n_tasks)?; + let (inner_plan, in_tasks) = dnode.as_ref().to_stage_info(n_tasks)?; // If the current stage has just 1 task, and the next stage is only going to have // 1 task, there's no point in having a network boundary in between, they can just // communicate in memory. if n_tasks == 1 && in_tasks == 1 { - let mut n = dnode.rollback()?; + let mut n = dnode.as_ref().rollback()?; if let Some(node) = n.as_any().downcast_ref::() { // Also trim PartitionIsolatorExec out of the plan. n = Arc::clone(node.children().first().unwrap()); } return Ok(Transformed::yes(n)); } - match Self::_distribute_plan_inner(query_id, inner_plan, num, depth + 1, in_tasks) { + match Self::_distribute_plan_inner(query_id, inner_plan.clone(), num, depth + 1, in_tasks) { Ok(v) => break v, Err(e) => match get_distribute_plan_err(&e) { None => return Err(e), @@ -265,13 +258,12 @@ impl DistributedPhysicalOptimizerRule { if in_tasks == *limit { return plan_err!("A node requested {limit} tasks for the stage its in, but that stage already has that many tasks"); } - dnode = dnode.with_input_tasks(*limit); + dnode = Referenced::Arced(dnode.as_ref().with_input_task_count(*limit)?); } }, } }; - let node = dnode.to_distributed(stage.num, &stage.plan)?; - inputs.push(stage); + let node = dnode.as_ref().with_input_stage(stage)?; Ok(Transformed::new(node, true, TreeNodeRecursion::Jump)) })?; @@ -282,11 +274,8 @@ impl DistributedPhysicalOptimizerRule { distributed.data = Arc::new(CoalescePartitionsExec::new(distributed.data)); } - let mut stage = StageExec::new(query_id, *num, distributed.data, inputs, n_tasks); + let stage = Stage::new(query_id, *num, distributed.data, n_tasks); *num += 1; - - stage.depth = depth; - Ok(stage) } } @@ -298,29 +287,25 @@ pub trait NetworkBoundary: ExecutionPlan { /// Returns the information necessary for building the next stage. /// - The head node of the stage. /// - the amount of tasks that stage will have. - fn to_stage_info( - &self, - n_tasks: usize, - ) -> Result<(Arc, usize), DataFusionError>; + fn to_stage_info(&self, n_tasks: usize) -> Result<(Arc, usize)>; /// re-assigns a different number of input tasks to the current [NetworkBoundary]. /// /// This will be called if upon building a stage, a [DistributedPlanError::LimitTasks] error /// is returned, prompting the [NetworkBoundary] to choose a different number of input tasks. - fn with_input_tasks(&self, input_tasks: usize) -> Arc; + fn with_input_task_count(&self, input_tasks: usize) -> Result>; - /// Called when a [StageExec] is correctly formed. The [NetworkBoundary] can use this + /// Called when a [Stage] is correctly formed. The [NetworkBoundary] can use this /// information to perform any internal transformations necessary for distributed execution. - fn to_distributed( - &self, - stage_num: usize, - stage_head: &Arc, - ) -> Result, DataFusionError>; + fn with_input_stage(&self, input_stage: Stage) -> Result>; + + /// Returns the assigned input [Stage], if any. + fn input_stage(&self) -> Option<&Stage>; /// The planner might decide to remove this [NetworkBoundary] from the plan if it decides that /// it's not going to bring any benefit. The [NetworkBoundary] will be replaced with whatever /// this function returns. - fn rollback(&self) -> Result, DataFusionError> { + fn rollback(&self) -> Result> { let children = self.children(); if children.len() != 1 { return plan_err!( @@ -333,6 +318,43 @@ pub trait NetworkBoundary: ExecutionPlan { } } +/// Extension trait for downcasting dynamic types to [NetworkBoundary]. +pub trait NetworkBoundaryExt { + /// Downcasts self to a [NetworkBoundary] if possible. + fn as_network_boundary(&self) -> Option<&dyn NetworkBoundary>; + /// Returns whether self is a [NetworkBoundary] or not. + fn is_network_boundary(&self) -> bool { + self.as_network_boundary().is_some() + } +} + +impl NetworkBoundaryExt for dyn ExecutionPlan { + fn as_network_boundary(&self) -> Option<&dyn NetworkBoundary> { + if let Some(node) = self.as_any().downcast_ref::() { + Some(node) + } else if let Some(node) = self.as_any().downcast_ref::() { + Some(node) + } else { + None + } + } +} + +/// Helper enum for storing either borrowed or owned trait object references +enum Referenced<'a, T: ?Sized> { + Borrowed(&'a T), + Arced(Arc), +} + +impl Referenced<'_, T> { + fn as_ref(&self) -> &T { + match self { + Self::Borrowed(r) => r, + Self::Arced(arc) => arc.as_ref(), + } + } +} + /// Error thrown during distributed planning that prompts the planner to change something and /// try again. #[derive(Debug)] @@ -369,12 +391,11 @@ fn get_distribute_plan_err(err: &DataFusionError) -> Option<&DistributedPlanErro #[cfg(test)] mod tests { - use crate::assert_snapshot; use crate::distributed_physical_optimizer_rule::DistributedPhysicalOptimizerRule; use crate::test_utils::parquet::register_parquet_tables; + use crate::{assert_snapshot, display_plan_ascii}; use datafusion::error::DataFusionError; use datafusion::execution::SessionStateBuilder; - use datafusion::physical_plan::displayable; use datafusion::prelude::{SessionConfig, SessionContext}; use std::sync::Arc; @@ -409,7 +430,7 @@ mod tests { let query = r#"SELECT * FROM weather"#; let plan = sql_to_explain(query, 1).await.unwrap(); assert_snapshot!(plan, @r" - ┌───── Stage 1 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ CoalescePartitionsExec │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, MaxTemp, Rainfall, Evaporation, Sunshine, WindGustDir, WindGustSpeed, WindDir9am, WindDir3pm, WindSpeed9am, WindSpeed3pm, Humidity9am, Humidity3pm, Pressure9am, Pressure3pm, Cloud9am, Cloud3pm, Temp9am, Temp3pm, RainToday, RISK_MM, RainTomorrow], file_type=parquet └────────────────────────────────────────────────── @@ -422,23 +443,23 @@ mod tests { r#"SELECT count(*), "RainToday" FROM weather GROUP BY "RainToday" ORDER BY count(*)"#; let plan = sql_to_explain(query, 2).await.unwrap(); assert_snapshot!(plan, @r" - ┌───── Stage 3 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[count(*)@0 as count(*), RainToday@1 as RainToday] │ SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] - │ NetworkCoalesceExec read_from=Stage 2, output_partitions=8, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 Tasks: t0:[p0,p1,p2,p3] t1:[p0,p1,p2,p3] + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3] t1:[p0,p1,p2,p3] │ SortExec: expr=[count(*)@0 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday, count(Int64(1))@1 as count(Int64(1))] │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 1, output_partitions=4, n_tasks=2, input_tasks=2 + │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7] t1:[p0,p1,p2,p3,p4,p5,p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7] t1:[p0,p1,p2,p3,p4,p5,p6,p7] │ RepartitionExec: partitioning=Hash([RainToday@0], 8), input_partitions=4 │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__] t1:[__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__] t1:[__,__,p0] │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday], file_type=parquet └────────────────────────────────────────────────── "); @@ -450,23 +471,23 @@ mod tests { r#"SELECT count(*), "RainToday" FROM weather GROUP BY "RainToday" ORDER BY count(*)"#; let plan = sql_to_explain(query, 2).await.unwrap(); assert_snapshot!(plan, @r" - ┌───── Stage 3 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[count(*)@0 as count(*), RainToday@1 as RainToday] │ SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] - │ NetworkCoalesceExec read_from=Stage 2, output_partitions=8, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 Tasks: t0:[p0,p1,p2,p3] t1:[p0,p1,p2,p3] + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3] t1:[p0,p1,p2,p3] │ SortExec: expr=[count(*)@0 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday, count(Int64(1))@1 as count(Int64(1))] │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 1, output_partitions=4, n_tasks=2, input_tasks=2 + │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7] t1:[p0,p1,p2,p3,p4,p5,p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7] t1:[p0,p1,p2,p3,p4,p5,p6,p7] │ RepartitionExec: partitioning=Hash([RainToday@0], 8), input_partitions=4 │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__] t1:[__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__] t1:[__,__,p0] │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday], file_type=parquet └────────────────────────────────────────────────── "); @@ -477,7 +498,7 @@ mod tests { let query = r#"SELECT a."MinTemp", b."MaxTemp" FROM weather a LEFT JOIN weather b ON a."RainToday" = b."RainToday" "#; let plan = sql_to_explain(query, 2).await.unwrap(); assert_snapshot!(plan, @r" - ┌───── Stage 1 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ CoalescePartitionsExec │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(RainToday@1, RainToday@1)], projection=[MinTemp@0, MaxTemp@2] @@ -516,39 +537,39 @@ mod tests { "#; let plan = sql_to_explain(query, 2).await.unwrap(); assert_snapshot!(plan, @r" - ┌───── Stage 4 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ CoalescePartitionsExec │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(RainTomorrow@1, RainTomorrow@1)], projection=[MinTemp@0, MaxTemp@2] │ CoalescePartitionsExec - │ NetworkCoalesceExec read_from=Stage 2, output_partitions=8, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=2 │ ProjectionExec: expr=[avg(weather.MaxTemp)@1 as MaxTemp, RainTomorrow@0 as RainTomorrow] │ AggregateExec: mode=FinalPartitioned, gby=[RainTomorrow@0 as RainTomorrow], aggr=[avg(weather.MaxTemp)] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 3, output_partitions=4, n_tasks=1, input_tasks=2 + │ [Stage 3] => NetworkShuffleExec: output_partitions=4, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 Tasks: t0:[p0,p1,p2,p3] t1:[p0,p1,p2,p3] + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3] t1:[p0,p1,p2,p3] │ ProjectionExec: expr=[avg(weather.MinTemp)@1 as MinTemp, RainTomorrow@0 as RainTomorrow] │ AggregateExec: mode=FinalPartitioned, gby=[RainTomorrow@0 as RainTomorrow], aggr=[avg(weather.MinTemp)] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 1, output_partitions=4, n_tasks=2, input_tasks=2 + │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7] t1:[p0,p1,p2,p3,p4,p5,p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7] t1:[p0,p1,p2,p3,p4,p5,p6,p7] │ RepartitionExec: partitioning=Hash([RainTomorrow@0], 8), input_partitions=4 │ AggregateExec: mode=Partial, gby=[RainTomorrow@1 as RainTomorrow], aggr=[avg(weather.MinTemp)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: RainToday@1 = yes, projection=[MinTemp@0, RainTomorrow@2] │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__] t1:[__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__] t1:[__,__,p0] │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday, RainTomorrow], file_type=parquet, predicate=RainToday@1 = yes, pruning_predicate=RainToday_null_count@2 != row_count@3 AND RainToday_min@0 <= yes AND yes <= RainToday_max@1, required_guarantees=[RainToday in (yes)] └────────────────────────────────────────────────── - ┌───── Stage 3 Tasks: t0:[p0,p1,p2,p3] t1:[p0,p1,p2,p3] + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3] t1:[p0,p1,p2,p3] │ RepartitionExec: partitioning=Hash([RainTomorrow@0], 4), input_partitions=4 │ AggregateExec: mode=Partial, gby=[RainTomorrow@1 as RainTomorrow], aggr=[avg(weather.MaxTemp)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: RainToday@1 = no, projection=[MaxTemp@0, RainTomorrow@2] │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__] t1:[__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__] t1:[__,__,p0] │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday, RainTomorrow], file_type=parquet, predicate=RainToday@1 = no, pruning_predicate=RainToday_null_count@2 != row_count@3 AND RainToday_min@0 <= no AND no <= RainToday_max@1, required_guarantees=[RainToday in (no)] └────────────────────────────────────────────────── "); @@ -559,13 +580,13 @@ mod tests { let query = r#"SELECT * FROM weather ORDER BY "MinTemp" DESC "#; let plan = sql_to_explain(query, 2).await.unwrap(); assert_snapshot!(plan, @r" - ┌───── Stage 2 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [MinTemp@0 DESC] - │ NetworkCoalesceExec read_from=Stage 1, output_partitions=4, input_tasks=2 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=4, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1] t1:[p2,p3] + ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] │ SortExec: expr=[MinTemp@0 DESC], preserve_partitioning=[true] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__] t1:[__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__] t1:[__,__,p0] │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, MaxTemp, Rainfall, Evaporation, Sunshine, WindGustDir, WindGustSpeed, WindDir9am, WindDir3pm, WindSpeed9am, WindSpeed3pm, Humidity9am, Humidity3pm, Pressure9am, Pressure3pm, Cloud9am, Cloud3pm, Temp9am, Temp3pm, RainToday, RISK_MM, RainTomorrow], file_type=parquet └────────────────────────────────────────────────── "); @@ -576,20 +597,20 @@ mod tests { let query = r#"SELECT DISTINCT "RainToday", "WindGustDir" FROM weather"#; let plan = sql_to_explain(query, 2).await.unwrap(); assert_snapshot!(plan, @r" - ┌───── Stage 3 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ CoalescePartitionsExec - │ NetworkCoalesceExec read_from=Stage 2, output_partitions=8, input_tasks=2 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 Tasks: t0:[p0,p1,p2,p3] t1:[p0,p1,p2,p3] + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3] t1:[p0,p1,p2,p3] │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 1, output_partitions=4, n_tasks=2, input_tasks=2 + │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7] t1:[p0,p1,p2,p3,p4,p5,p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7] t1:[p0,p1,p2,p3,p4,p5,p6,p7] │ RepartitionExec: partitioning=Hash([RainToday@0, WindGustDir@1], 8), input_partitions=4 │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__] t1:[__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__] t1:[__,__,p0] │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday, WindGustDir], file_type=parquet └────────────────────────────────────────────────── "); @@ -623,8 +644,6 @@ mod tests { let df = ctx.sql(query).await?; let physical_plan = df.create_physical_plan().await?; - let display = displayable(physical_plan.as_ref()).indent(true).to_string(); - - Ok(display) + Ok(display_plan_ascii(physical_plan.as_ref())) } } diff --git a/src/common/partitioning.rs b/src/execution_plans/common.rs similarity index 58% rename from src/common/partitioning.rs rename to src/execution_plans/common.rs index 538a725..7f55eee 100644 --- a/src/common/partitioning.rs +++ b/src/execution_plans/common.rs @@ -1,7 +1,18 @@ +use datafusion::common::{DataFusionError, plan_err}; use datafusion::physical_expr::Partitioning; -use datafusion::physical_plan::PlanProperties; +use datafusion::physical_plan::{ExecutionPlan, PlanProperties}; +use std::sync::Arc; -pub fn scale_partitioning_props( +pub(super) fn require_one_child( + children: &[Arc], +) -> Result, DataFusionError> { + if children.len() != 1 { + return plan_err!("Expected exactly 1 children, got {}", children.len()); + } + Ok(children[0].clone()) +} + +pub(super) fn scale_partitioning_props( props: &PlanProperties, f: impl FnOnce(usize) -> usize, ) -> PlanProperties { @@ -13,7 +24,7 @@ pub fn scale_partitioning_props( ) } -pub fn scale_partitioning( +pub(super) fn scale_partitioning( partitioning: &Partitioning, f: impl FnOnce(usize) -> usize, ) -> Partitioning { diff --git a/src/execution_plans/distributed.rs b/src/execution_plans/distributed.rs new file mode 100644 index 0000000..87d7e62 --- /dev/null +++ b/src/execution_plans/distributed.rs @@ -0,0 +1,126 @@ +use crate::channel_resolver_ext::get_distributed_channel_resolver; +use crate::distributed_physical_optimizer_rule::NetworkBoundaryExt; +use crate::execution_plans::common::require_one_child; +use crate::protobuf::DistributedCodec; +use crate::{ExecutionTask, Stage}; +use datafusion::common::exec_err; +use datafusion::common::tree_node::{Transformed, TreeNode}; +use datafusion::execution::{SendableRecordBatchStream, TaskContext}; +use datafusion::physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties}; +use datafusion_proto::physical_plan::PhysicalExtensionCodec; +use rand::Rng; +use std::any::Any; +use std::fmt::Formatter; +use std::sync::Arc; +use url::Url; + +/// [ExecutionPlan] that executes the inner plan in distributed mode. +/// Before executing it, two modifications are lazily performed on the plan: +/// 1. Assigns worker URLs to all the stages. A random set of URLs are sampled from the +/// channel resolver and assigned to each task in each stage. +/// 2. Encodes all the plans in protobuf format so that network boundary nodes can send them +/// over the wire. +#[derive(Debug, Clone)] +pub struct DistributedExec { + pub plan: Arc, +} + +impl DistributedExec { + pub fn new(plan: Arc) -> Self { + Self { plan } + } + + fn prepare_plan( + &self, + urls: &[Url], + codec: &dyn PhysicalExtensionCodec, + ) -> datafusion::common::Result> { + let prepared = Arc::clone(&self.plan).transform_up(|plan| { + let Some(plan) = plan.as_network_boundary() else { + return Ok(Transformed::no(plan)); + }; + + let mut rng = rand::thread_rng(); + let start_idx = rng.gen_range(0..urls.len()); + + let Some(stage) = plan.input_stage() else { + return exec_err!( + "NetworkBoundary '{}' has not been assigned a stage", + plan.name() + ); + }; + + let ready_stage = Stage { + query_id: stage.query_id, + num: stage.num, + plan: stage.plan.to_encoded(codec)?, + tasks: stage + .tasks + .iter() + .enumerate() + .map(|(i, _)| ExecutionTask { + url: Some(urls[(start_idx + i) % urls.len()].clone()), + }) + .collect::>(), + }; + + Ok(Transformed::yes(plan.with_input_stage(ready_stage)?)) + })?; + Ok(prepared.data) + } +} + +impl DisplayAs for DistributedExec { + fn fmt_as(&self, _: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { + write!(f, "DistributedExec") + } +} + +impl ExecutionPlan for DistributedExec { + fn name(&self) -> &str { + "DistributedExec" + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn properties(&self) -> &PlanProperties { + self.plan.properties() + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.plan] + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> datafusion::common::Result> { + Ok(Arc::new(DistributedExec { + plan: require_one_child(&children)?, + })) + } + + fn execute( + &self, + partition: usize, + context: Arc, + ) -> datafusion::common::Result { + if partition > 0 { + // The DistributedExec node calls try_assign_urls() lazily upon calling .execute(). This means + // that .execute() must only be called once, as we cannot afford to perform several + // random URL assignation while calling multiple partitions, as they will differ, + // producing an invalid plan + return exec_err!( + "DistributedExec must only have 1 partition, but it was called with partition index {partition}" + ); + } + + let channel_resolver = get_distributed_channel_resolver(context.session_config())?; + let codec = DistributedCodec::new_combined_with_user(context.session_config()); + + let plan = self.prepare_plan(&channel_resolver.get_urls()?, &codec)?; + plan.execute(partition, context) + } +} diff --git a/src/execution_plans/mod.rs b/src/execution_plans/mod.rs index bf421e6..6a64763 100644 --- a/src/execution_plans/mod.rs +++ b/src/execution_plans/mod.rs @@ -1,13 +1,12 @@ +mod common; +mod distributed; mod metrics; mod network_coalesce; mod network_shuffle; mod partition_isolator; -mod stage; +pub use distributed::DistributedExec; pub use metrics::MetricsWrapperExec; pub use network_coalesce::{NetworkCoalesceExec, NetworkCoalesceReady}; pub use network_shuffle::{NetworkShuffleExec, NetworkShuffleReadyExec}; pub use partition_isolator::PartitionIsolatorExec; -pub(crate) use stage::InputStage; -pub use stage::display_plan_graphviz; -pub use stage::{DistributedTaskContext, ExecutionTask, StageExec}; diff --git a/src/execution_plans/network_coalesce.rs b/src/execution_plans/network_coalesce.rs index b3f0ad2..a450c39 100644 --- a/src/execution_plans/network_coalesce.rs +++ b/src/execution_plans/network_coalesce.rs @@ -1,19 +1,19 @@ -use crate::ChannelResolver; use crate::channel_resolver_ext::get_distributed_channel_resolver; -use crate::common::scale_partitioning_props; use crate::config_extension_ext::ContextGrpcMetadata; use crate::distributed_physical_optimizer_rule::{NetworkBoundary, limit_tasks_err}; -use crate::execution_plans::{DistributedTaskContext, StageExec}; +use crate::execution_plans::common::{require_one_child, scale_partitioning_props}; use crate::flight_service::DoGet; use crate::metrics::MetricsCollectingStream; use crate::metrics::proto::MetricsSetProto; -use crate::protobuf::{DistributedCodec, StageKey, proto_from_input_stage}; -use crate::protobuf::{map_flight_to_datafusion_error, map_status_to_datafusion_error}; +use crate::protobuf::{StageKey, map_flight_to_datafusion_error, map_status_to_datafusion_error}; +use crate::stage::MaybeEncodedPlan; +use crate::{ChannelResolver, DistributedTaskContext, Stage}; use arrow_flight::Ticket; use arrow_flight::decode::FlightRecordBatchStream; use arrow_flight::error::FlightError; +use bytes::Bytes; use dashmap::DashMap; -use datafusion::common::{exec_err, internal_datafusion_err, internal_err, plan_err}; +use datafusion::common::{exec_err, internal_err, plan_err}; use datafusion::datasource::schema_adapter::DefaultSchemaAdapterFactory; use datafusion::error::DataFusionError; use datafusion::execution::{SendableRecordBatchStream, TaskContext}; @@ -86,8 +86,7 @@ pub struct NetworkCoalescePending { pub struct NetworkCoalesceReady { /// the properties we advertise for this execution plan pub(crate) properties: PlanProperties, - pub(crate) stage_num: usize, - pub(crate) input_tasks: usize, + pub(crate) input_stage: Stage, /// metrics_collection is used to collect metrics from child tasks. It is empty when an /// is instantiated (deserialized, created via [NetworkCoalesceExec::new_ready] etc...). /// Metrics are populated in this map via [NetworkCoalesceExec::execute]. @@ -128,6 +127,7 @@ impl NetworkBoundary for NetworkCoalesceExec { return plan_err!("can only return wrapped child if on Pending state"); }; + // As this node coalesces multiple tasks into 1, it must run in a stage with 1 task. if n_tasks > 1 { return Err(limit_tasks_err(1)); } @@ -135,51 +135,66 @@ impl NetworkBoundary for NetworkCoalesceExec { Ok((Arc::clone(&pending.child), pending.input_tasks)) } - fn to_distributed( + fn with_input_stage( &self, - stage_num: usize, - stage_head: &Arc, + input_stage: Stage, ) -> Result, DataFusionError> { - let NetworkCoalesceExec::Pending(pending) = self else { - return internal_err!("NetworkCoalesceExec is already distributed"); - }; - - let ready = NetworkCoalesceReady { - properties: scale_partitioning_props(stage_head.properties(), |p| { - p * pending.input_tasks - }), - stage_num, - input_tasks: pending.input_tasks, - metrics_collection: Default::default(), - }; + match self { + Self::Pending(pending) => { + let properties = input_stage.plan.decoded()?.properties(); + let ready = NetworkCoalesceReady { + properties: scale_partitioning_props(properties, |p| p * pending.input_tasks), + input_stage, + metrics_collection: Default::default(), + }; + + Ok(Arc::new(Self::Ready(ready))) + } + Self::Ready(ready) => { + let mut ready = ready.clone(); + ready.input_stage = input_stage; + Ok(Arc::new(Self::Ready(ready))) + } + } + } - Ok(Arc::new(Self::Ready(ready))) + fn input_stage(&self) -> Option<&Stage> { + match self { + Self::Pending(_) => None, + Self::Ready(v) => Some(&v.input_stage), + } } - fn with_input_tasks(&self, input_tasks: usize) -> Arc { - Arc::new(match self { - NetworkCoalesceExec::Pending(pending) => { - NetworkCoalesceExec::Pending(NetworkCoalescePending { - properties: pending.properties.clone(), - input_tasks, - child: pending.child.clone(), - }) - } - NetworkCoalesceExec::Ready(ready) => NetworkCoalesceExec::Ready(NetworkCoalesceReady { - properties: scale_partitioning_props(&ready.properties, |p| { - p * input_tasks / ready.input_tasks - }), - stage_num: ready.stage_num, + fn with_input_task_count( + &self, + input_tasks: usize, + ) -> Result, DataFusionError> { + Ok(Arc::new(match self { + Self::Pending(pending) => Self::Pending(NetworkCoalescePending { + properties: pending.properties.clone(), input_tasks, - metrics_collection: Arc::clone(&ready.metrics_collection), + child: pending.child.clone(), }), - }) + Self::Ready(_) => { + plan_err!("Self can only re-assign input tasks if in 'Pending' state")? + } + })) } } impl DisplayAs for NetworkCoalesceExec { fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { - write!(f, "NetworkCoalesceExec") + let Self::Ready(self_ready) = self else { + return write!(f, "NetworkCoalesceExec"); + }; + + let input_tasks = self_ready.input_stage.tasks.len(); + let partitions = self_ready.properties.partitioning.partition_count(); + let stage = self_ready.input_stage.num; + write!( + f, + "[Stage {stage}] => NetworkCoalesceExec: output_partitions={partitions}, input_tasks={input_tasks}", + ) } } @@ -202,7 +217,10 @@ impl ExecutionPlan for NetworkCoalesceExec { fn children(&self) -> Vec<&Arc> { match self { NetworkCoalesceExec::Pending(v) => vec![&v.child], - NetworkCoalesceExec::Ready(_) => vec![], + NetworkCoalesceExec::Ready(v) => match &v.input_stage.plan { + MaybeEncodedPlan::Decoded(v) => vec![v], + MaybeEncodedPlan::Encoded(_) => vec![], + }, } } @@ -210,13 +228,18 @@ impl ExecutionPlan for NetworkCoalesceExec { self: Arc, children: Vec>, ) -> Result, DataFusionError> { - if !children.is_empty() { - return plan_err!( - "NetworkCoalesceExec: wrong number of children, expected 0, got {}", - children.len() - ); + match self.as_ref() { + Self::Pending(v) => { + let mut v = v.clone(); + v.child = require_one_child(&children)?; + Ok(Arc::new(Self::Pending(v))) + } + Self::Ready(v) => { + let mut v = v.clone(); + v.input_stage.plan = MaybeEncodedPlan::Decoded(require_one_child(&children)?); + Ok(Arc::new(Self::Ready(v))) + } } - Ok(self) } fn execute( @@ -233,16 +256,8 @@ impl ExecutionPlan for NetworkCoalesceExec { // get the channel manager and current stage from our context let channel_resolver = get_distributed_channel_resolver(context.session_config())?; - // the `NetworkCoalesceExec` node can only be executed in the context of a `StageExec` - let stage = StageExec::from_ctx(&context)?; - - // of our input stages find the one that we are supposed to be reading from - let input_stage = stage.input_stage(self_ready.stage_num)?; - - let codec = DistributedCodec::new_combined_with_user(context.session_config()); - let input_stage_proto = proto_from_input_stage(input_stage, &codec).map_err(|e| { - internal_datafusion_err!("NetworkCoalesceExec: failed to convert stage to proto: {e}") - })?; + let input_stage = &self_ready.input_stage; + let encoded_input_plan = input_stage.plan.encoded()?; let context_headers = ContextGrpcMetadata::headers_from_ctx(&context); let task_context = DistributedTaskContext::from_ctx(&context); @@ -251,7 +266,7 @@ impl ExecutionPlan for NetworkCoalesceExec { } let partitions_per_task = - self.properties().partitioning.partition_count() / input_stage.tasks().len(); + self.properties().partitioning.partition_count() / input_stage.tasks.len(); let target_task = partition / partitions_per_task; let target_partition = partition % partitions_per_task; @@ -261,21 +276,22 @@ impl ExecutionPlan for NetworkCoalesceExec { Extensions::default(), Ticket { ticket: DoGet { - stage_proto: input_stage_proto, + plan_proto: encoded_input_plan.clone(), target_partition: target_partition as u64, stage_key: Some(StageKey { - query_id: stage.query_id.to_string(), - stage_id: input_stage.num() as u64, + query_id: Bytes::from(input_stage.query_id.as_bytes().to_vec()), + stage_id: input_stage.num as u64, task_number: target_task as u64, }), target_task_index: target_task as u64, + target_task_count: input_stage.tasks.len() as u64, } .encode_to_vec() .into(), }, ); - let Some(task) = input_stage.tasks().get(target_task) else { + let Some(task) = input_stage.tasks.get(target_task) else { return internal_err!("ProgrammingError: Task {target_task} not found"); }; diff --git a/src/execution_plans/network_shuffle.rs b/src/execution_plans/network_shuffle.rs index 51beeb0..576f8cb 100644 --- a/src/execution_plans/network_shuffle.rs +++ b/src/execution_plans/network_shuffle.rs @@ -1,19 +1,20 @@ -use crate::ChannelResolver; use crate::channel_resolver_ext::get_distributed_channel_resolver; -use crate::common::scale_partitioning; use crate::config_extension_ext::ContextGrpcMetadata; use crate::distributed_physical_optimizer_rule::NetworkBoundary; -use crate::execution_plans::{DistributedTaskContext, StageExec}; +use crate::execution_plans::common::{require_one_child, scale_partitioning}; use crate::flight_service::DoGet; use crate::metrics::MetricsCollectingStream; use crate::metrics::proto::MetricsSetProto; -use crate::protobuf::{DistributedCodec, StageKey, proto_from_input_stage}; +use crate::protobuf::StageKey; use crate::protobuf::{map_flight_to_datafusion_error, map_status_to_datafusion_error}; +use crate::stage::MaybeEncodedPlan; +use crate::{ChannelResolver, DistributedTaskContext, Stage}; use arrow_flight::Ticket; use arrow_flight::decode::FlightRecordBatchStream; use arrow_flight::error::FlightError; +use bytes::Bytes; use dashmap::DashMap; -use datafusion::common::{exec_err, internal_datafusion_err, internal_err, plan_err}; +use datafusion::common::{exec_err, internal_datafusion_err, plan_err}; use datafusion::datasource::schema_adapter::DefaultSchemaAdapterFactory; use datafusion::error::DataFusionError; use datafusion::execution::{SendableRecordBatchStream, TaskContext}; @@ -135,7 +136,7 @@ pub struct NetworkShufflePendingExec { pub struct NetworkShuffleReadyExec { /// the properties we advertise for this execution plan pub(crate) properties: PlanProperties, - pub(crate) stage_num: usize, + pub(crate) input_stage: Stage, /// metrics_collection is used to collect metrics from child tasks. It is empty when an /// is instantiated (deserialized, created via [NetworkShuffleExec::new_ready] etc...). /// Metrics are populated in this map via [NetworkShuffleExec::execute]. @@ -198,44 +199,63 @@ impl NetworkBoundary for NetworkShuffleExec { Ok((next_stage_plan, pending.input_tasks)) } - fn with_input_tasks(&self, input_tasks: usize) -> Arc { - Arc::new(match self { - NetworkShuffleExec::Pending(prev) => { - NetworkShuffleExec::Pending(NetworkShufflePendingExec { - repartition_exec: Arc::clone(&prev.repartition_exec), - input_tasks, - }) - } - NetworkShuffleExec::Ready(prev) => NetworkShuffleExec::Ready(NetworkShuffleReadyExec { - properties: prev.properties.clone(), - stage_num: prev.stage_num, - metrics_collection: Arc::clone(&prev.metrics_collection), + fn with_input_task_count( + &self, + input_tasks: usize, + ) -> Result, DataFusionError> { + Ok(Arc::new(match self { + Self::Pending(prev) => Self::Pending(NetworkShufflePendingExec { + repartition_exec: Arc::clone(&prev.repartition_exec), + input_tasks, }), - }) + Self::Ready(_) => plan_err!( + "NetworkShuffleExec can only re-assign input tasks if in 'Pending' state" + )?, + })) } - fn to_distributed( + fn with_input_stage( &self, - stage_num: usize, - _stage_head: &Arc, + input_stage: Stage, ) -> Result, DataFusionError> { - let NetworkShuffleExec::Pending(pending) = self else { - return internal_err!("NetworkShuffleExec is already distributed"); - }; - - let ready = NetworkShuffleReadyExec { - properties: pending.repartition_exec.properties().clone(), - stage_num, - metrics_collection: Default::default(), - }; + match self { + Self::Pending(pending) => { + let ready = NetworkShuffleReadyExec { + properties: pending.repartition_exec.properties().clone(), + input_stage, + metrics_collection: Default::default(), + }; + Ok(Arc::new(Self::Ready(ready))) + } + Self::Ready(ready) => { + let mut ready = ready.clone(); + ready.input_stage = input_stage; + Ok(Arc::new(Self::Ready(ready))) + } + } + } - Ok(Arc::new(Self::Ready(ready))) + fn input_stage(&self) -> Option<&Stage> { + match self { + Self::Pending(_) => None, + Self::Ready(v) => Some(&v.input_stage), + } } } impl DisplayAs for NetworkShuffleExec { fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { - write!(f, "NetworkShuffleExec") + let Self::Ready(self_ready) = self else { + return write!(f, "NetworkShuffleExec: Pending"); + }; + + let input_tasks = self_ready.input_stage.tasks.len(); + let partitions = self_ready.properties.partitioning.partition_count(); + let stage = self_ready.input_stage.num; + write!( + f, + "[Stage {stage}] => NetworkShuffleExec: output_partitions={partitions}, input_tasks={input_tasks}", + ) } } @@ -258,7 +278,10 @@ impl ExecutionPlan for NetworkShuffleExec { fn children(&self) -> Vec<&Arc> { match self { NetworkShuffleExec::Pending(v) => vec![&v.repartition_exec], - NetworkShuffleExec::Ready(_) => vec![], + NetworkShuffleExec::Ready(v) => match &v.input_stage.plan { + MaybeEncodedPlan::Decoded(v) => vec![v], + MaybeEncodedPlan::Encoded(_) => vec![], + }, } } @@ -266,13 +289,18 @@ impl ExecutionPlan for NetworkShuffleExec { self: Arc, children: Vec>, ) -> Result, DataFusionError> { - if !children.is_empty() { - return plan_err!( - "NetworkShuffleExec: wrong number of children, expected 0, got {}", - children.len() - ); + match self.as_ref() { + Self::Pending(v) => { + let mut v = v.clone(); + v.repartition_exec = require_one_child(&children)?; + Ok(Arc::new(Self::Pending(v))) + } + Self::Ready(v) => { + let mut v = v.clone(); + v.input_stage.plan = MaybeEncodedPlan::Decoded(require_one_child(&children)?); + Ok(Arc::new(Self::Ready(v))) + } } - Ok(self) } fn execute( @@ -289,21 +317,13 @@ impl ExecutionPlan for NetworkShuffleExec { // get the channel manager and current stage from our context let channel_resolver = get_distributed_channel_resolver(context.session_config())?; - // the `NetworkShuffleExec` node can only be executed in the context of a `StageExec` - let stage = StageExec::from_ctx(&context)?; - - // of our child stages find the one that matches the one we are supposed to be - // reading from - let input_stage = stage.input_stage(self_ready.stage_num)?; - - let codec = DistributedCodec::new_combined_with_user(context.session_config()); - let input_stage_proto = proto_from_input_stage(input_stage, &codec).map_err(|e| { - internal_datafusion_err!("NetworkShuffleExec: failed to convert stage to proto: {e}") - })?; + let input_stage = &self_ready.input_stage; + let encoded_input_plan = input_stage.plan.encoded()?; - let input_stage_tasks = input_stage.tasks().to_vec(); - let input_stage_num = input_stage.num() as u64; - let query_id = stage.query_id.to_string(); + let input_stage_tasks = input_stage.tasks.to_vec(); + let input_task_count = input_stage_tasks.len(); + let input_stage_num = input_stage.num as u64; + let query_id = Bytes::from(input_stage.query_id.as_bytes().to_vec()); let context_headers = ContextGrpcMetadata::headers_from_ctx(&context); let task_context = DistributedTaskContext::from_ctx(&context); @@ -321,7 +341,7 @@ impl ExecutionPlan for NetworkShuffleExec { Extensions::default(), Ticket { ticket: DoGet { - stage_proto: input_stage_proto.clone(), + plan_proto: encoded_input_plan.clone(), target_partition: (off + partition) as u64, stage_key: Some(StageKey { query_id: query_id.clone(), @@ -329,6 +349,7 @@ impl ExecutionPlan for NetworkShuffleExec { task_number: i as u64, }), target_task_index: i as u64, + target_task_count: input_task_count as u64, } .encode_to_vec() .into(), diff --git a/src/execution_plans/partition_isolator.rs b/src/execution_plans/partition_isolator.rs index 987f5f5..81dd67c 100644 --- a/src/execution_plans/partition_isolator.rs +++ b/src/execution_plans/partition_isolator.rs @@ -1,6 +1,5 @@ -use crate::StageExec; +use crate::DistributedTaskContext; use crate::distributed_physical_optimizer_rule::limit_tasks_err; -use crate::execution_plans::DistributedTaskContext; use datafusion::common::{exec_err, plan_err}; use datafusion::error::DataFusionError; use datafusion::execution::TaskContext; @@ -138,7 +137,27 @@ impl PartitionIsolatorExec { impl DisplayAs for PartitionIsolatorExec { fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { - write!(f, "PartitionIsolatorExec",) + let PartitionIsolatorExec::Ready(self_ready) = self else { + return write!(f, "PartitionIsolatorExec"); + }; + let input_partitions = self.input().output_partitioning().partition_count(); + let partition_groups = + PartitionIsolatorExec::partition_groups(input_partitions, self_ready.n_tasks); + + let n: usize = partition_groups.iter().map(|v| v.len()).sum(); + let mut partitions = vec![]; + for _ in 0..self_ready.n_tasks { + partitions.push(vec!["__".to_string(); n]); + } + + write!(f, "PartitionIsolatorExec: ")?; + for (i, partition_group) in partition_groups.iter().enumerate() { + for (j, p) in partition_group.iter().enumerate() { + partitions[i][*p] = format!("p{j}") + } + write!(f, "t{i}:[{}] ", partitions[i].join(","))?; + } + Ok(()) } } @@ -191,12 +210,14 @@ impl ExecutionPlan for PartitionIsolatorExec { }; let task_context = DistributedTaskContext::from_ctx(&context); - let stage = StageExec::from_ctx(&context)?; let input_partitions = self_ready.input.output_partitioning().partition_count(); - let partition_group = - Self::partition_group(input_partitions, task_context.task_index, stage.tasks.len()); + let partition_group = Self::partition_group( + input_partitions, + task_context.task_index, + task_context.task_count, + ); // if our partition group is [7,8,9] and we are asked for parittion 1, // then look up that index in our group and execute that partition, in this diff --git a/src/flight_service/do_get.rs b/src/flight_service/do_get.rs index 8b754de..17c9c26 100644 --- a/src/flight_service/do_get.rs +++ b/src/flight_service/do_get.rs @@ -1,13 +1,13 @@ +use crate::DistributedTaskContext; use crate::common::map_last_stream; use crate::config_extension_ext::ContextGrpcMetadata; -use crate::execution_plans::{DistributedTaskContext, StageExec}; use crate::flight_service::service::ArrowFlightEndpoint; use crate::flight_service::session_builder::DistributedSessionBuilderContext; use crate::metrics::TaskMetricsCollector; use crate::metrics::proto::df_metrics_set_to_proto; use crate::protobuf::{ AppMetadata, DistributedCodec, FlightAppMetadata, MetricsCollection, StageKey, TaskMetrics, - datafusion_error_to_tonic_status, stage_from_proto, + datafusion_error_to_tonic_status, }; use arrow_flight::FlightData; use arrow_flight::Ticket; @@ -17,7 +17,11 @@ use arrow_flight::flight_service_server::FlightService; use bytes::Bytes; use datafusion::common::exec_datafusion_err; +use datafusion::error::DataFusionError; +use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::SessionContext; +use datafusion_proto::physical_plan::AsExecutionPlan; +use datafusion_proto::protobuf::PhysicalPlanNode; use futures::TryStreamExt; use prost::Message; use std::sync::Arc; @@ -26,20 +30,22 @@ use tonic::{Request, Response, Status}; #[derive(Clone, PartialEq, ::prost::Message)] pub struct DoGet { - /// The [StageExec] we are going to execute encoded as protobuf bytes. + /// The [Arc] we are going to execute encoded as protobuf bytes. #[prost(bytes, tag = "1")] - pub stage_proto: Bytes, + pub plan_proto: Bytes, /// The index to the task within the stage that we want to execute #[prost(uint64, tag = "2")] pub target_task_index: u64, - /// the partition number we want to execute #[prost(uint64, tag = "3")] + pub target_task_count: u64, + /// the partition number we want to execute + #[prost(uint64, tag = "4")] pub target_partition: u64, /// The stage key that identifies the stage. This is useful to keep /// outside of the stage proto as it is used to store the stage /// and we may not need to deserialize the entire stage proto /// if we already have stored it - #[prost(message, optional, tag = "4")] + #[prost(message, optional, tag = "5")] pub stage_key: Option, } @@ -47,7 +53,7 @@ pub struct DoGet { /// TaskData stores state for a single task being executed by this Endpoint. It may be shared /// by concurrent requests for the same task which execute separate partitions. pub struct TaskData { - pub(super) stage: Arc, + pub(super) plan: Arc, /// `num_partitions_remaining` is initialized to the total number of partitions in the task (not /// only tasks in the partition group). This is decremented for each request to the endpoint /// for this task. Once this count is zero, the task is likely complete. The task may not be @@ -87,33 +93,31 @@ impl ArrowFlightEndpoint { let stage_data = once .get_or_try_init(|| async { - let stage_proto = doget.stage_proto; - let stage = - stage_from_proto(stage_proto, &ctx, &self.runtime, &codec).map_err(|err| { - Status::invalid_argument(format!("Cannot decode stage proto: {err}")) - })?; + let proto_node = PhysicalPlanNode::try_decode(doget.plan_proto.as_ref())?; + let plan = proto_node.try_into_physical_plan(&ctx, &self.runtime, &codec)?; // Initialize partition count to the number of partitions in the stage - let total_partitions = stage.plan.properties().partitioning.partition_count(); - Ok::<_, Status>(TaskData { - stage: Arc::new(stage), + let total_partitions = plan.properties().partitioning.partition_count(); + Ok::<_, DataFusionError>(TaskData { + plan, num_partitions_remaining: Arc::new(AtomicUsize::new(total_partitions)), }) }) - .await?; - let stage = Arc::clone(&stage_data.stage); + .await + .map_err(|err| Status::invalid_argument(format!("Cannot decode stage proto: {err}")))?; + let plan = Arc::clone(&stage_data.plan); // Find out which partition group we are executing let cfg = session_state.config_mut(); - cfg.set_extension(Arc::clone(&stage)); cfg.set_extension(Arc::new(ContextGrpcMetadata(metadata.into_headers()))); - cfg.set_extension(Arc::new(DistributedTaskContext::new( - doget.target_task_index as usize, - ))); + cfg.set_extension(Arc::new(DistributedTaskContext { + task_index: doget.target_task_index as usize, + task_count: doget.target_task_count as usize, + })); - let partition_count = stage.plan.properties().partitioning.partition_count(); + let partition_count = plan.properties().partitioning.partition_count(); let target_partition = doget.target_partition as usize; - let plan_name = stage.plan.name(); + let plan_name = plan.name(); if target_partition >= partition_count { return Err(datafusion_error_to_tonic_status(&exec_datafusion_err!( "partition {target_partition} not available. The head plan {plan_name} of the stage just has {partition_count} partitions" @@ -123,8 +127,7 @@ impl ArrowFlightEndpoint { // Rather than executing the `StageExec` itself, we want to execute the inner plan instead, // as executing `StageExec` performs some worker assignation that should have already been // done in the head stage. - let stream = stage - .plan + let stream = plan .execute(doget.target_partition as usize, session_state.task_ctx()) .map_err(|err| Status::internal(format!("Error executing stage plan: {err:#?}")))?; @@ -141,7 +144,7 @@ impl ArrowFlightEndpoint { if num_partitions_remaining.fetch_sub(1, Ordering::SeqCst) == 1 { task_data_entries.remove(key.clone()); } - last.and_then(|el| collect_and_create_metrics_flight_data(key, stage, el)) + last.and_then(|el| collect_and_create_metrics_flight_data(key, plan, el)) }); Ok(Response::new(Box::pin(stream.map_err(|err| match err { @@ -158,12 +161,12 @@ fn missing(field: &'static str) -> impl FnOnce() -> Status { /// Collects metrics from the provided stage and includes it in the flight data fn collect_and_create_metrics_flight_data( stage_key: StageKey, - stage: Arc, + plan: Arc, incoming: FlightData, ) -> Result { // Get the metrics for the task executed on this worker. Separately, collect metrics for child tasks. let mut result = TaskMetricsCollector::new() - .collect(stage.plan.clone()) + .collect(plan) .map_err(|err| FlightError::ProtocolError(err.to_string()))?; // Add the metrics for this task into the collection of task metrics. @@ -208,7 +211,6 @@ mod tests { use super::*; use crate::ExecutionTask; use crate::flight_service::session_builder::DefaultSessionBuilder; - use crate::protobuf::proto_from_stage; use arrow::datatypes::{Schema, SchemaRef}; use arrow_flight::Ticket; use datafusion::physical_expr::Partitioning; @@ -230,50 +232,46 @@ mod tests { let num_tasks = 3; let num_partitions_per_task = 3; let stage_id = 1; - let query_id = Uuid::new_v4(); + let query_id = Bytes::from(Uuid::new_v4().into_bytes().to_vec()); // Set up protos. let mut tasks = Vec::new(); for _ in 0..num_tasks { tasks.push(ExecutionTask { url: None }); } - - let stage = StageExec { - query_id, - num: 1, - name: format!("test_stage_{}", 1), - plan: create_mock_physical_plan(num_partitions_per_task), - inputs: vec![], - tasks, - depth: 0, - }; + let plan = create_mock_physical_plan(num_partitions_per_task); + let plan_proto: Bytes = + PhysicalPlanNode::try_from_physical_plan(plan, &DefaultPhysicalExtensionCodec {}) + .unwrap() + .encode_to_vec() + .into(); let task_keys = [ StageKey { - query_id: query_id.to_string(), + query_id: query_id.clone(), stage_id, task_number: 0, }, StageKey { - query_id: query_id.to_string(), + query_id: query_id.clone(), stage_id, task_number: 1, }, StageKey { - query_id: query_id.to_string(), + query_id: query_id.clone(), stage_id, task_number: 2, }, ]; - let stage_proto = proto_from_stage(&stage, &DefaultPhysicalExtensionCodec {}).unwrap(); - let stage_proto_for_closure = stage_proto.clone(); + let plan_proto_for_closure = plan_proto.clone(); let endpoint_ref = &endpoint; let do_get = async move |partition: u64, task_number: u64, stage_key: StageKey| { - let stage_proto = stage_proto_for_closure.clone(); + let plan_proto = plan_proto_for_closure.clone(); let doget = DoGet { - stage_proto: stage_proto.encode_to_vec().into(), + plan_proto, target_task_index: task_number, + target_task_count: num_tasks, target_partition: partition, stage_key: Some(stage_key), }; @@ -300,7 +298,7 @@ mod tests { } // Check that the endpoint has not evicted any task states. - assert_eq!(endpoint.task_data_entries.len(), num_tasks); + assert_eq!(endpoint.task_data_entries.len(), num_tasks as usize); // Run the last partition of task 0. Any partition number works. Verify that the task state // is evicted because all partitions have been processed. diff --git a/src/lib.rs b/src/lib.rs index 9958379..59f2f2b 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -8,6 +8,7 @@ mod distributed_physical_optimizer_rule; mod execution_plans; mod flight_service; mod metrics; +mod stage; mod protobuf; #[cfg(any(feature = "integration", test))] @@ -15,14 +16,15 @@ pub mod test_utils; pub use channel_resolver_ext::{BoxCloneSyncChannel, ChannelResolver}; pub use distributed_ext::DistributedExt; -pub use distributed_physical_optimizer_rule::DistributedPhysicalOptimizerRule; -pub use execution_plans::display_plan_graphviz; -pub use execution_plans::{ - DistributedTaskContext, ExecutionTask, NetworkCoalesceExec, NetworkShuffleExec, - PartitionIsolatorExec, StageExec, +pub use distributed_physical_optimizer_rule::{ + DistributedPhysicalOptimizerRule, NetworkBoundaryExt, }; +pub use execution_plans::{NetworkCoalesceExec, NetworkShuffleExec, PartitionIsolatorExec}; pub use flight_service::{ ArrowFlightEndpoint, DefaultSessionBuilder, DistributedSessionBuilder, DistributedSessionBuilderContext, MappedDistributedSessionBuilder, MappedDistributedSessionBuilderExt, }; +pub use stage::{ + DistributedTaskContext, ExecutionTask, Stage, display_plan_ascii, display_plan_graphviz, +}; diff --git a/src/metrics/metrics_collecting_stream.rs b/src/metrics/metrics_collecting_stream.rs index 32a424d..7e2e4e6 100644 --- a/src/metrics/metrics_collecting_stream.rs +++ b/src/metrics/metrics_collecting_stream.rs @@ -137,12 +137,12 @@ mod tests { async fn test_metrics_collecting_stream_extracts_and_removes_metadata() { let stage_keys = vec![ StageKey { - query_id: "test_query".to_string(), + query_id: Bytes::from("test_query"), stage_id: 1, task_number: 1, }, StageKey { - query_id: "test_query".to_string(), + query_id: Bytes::from("test_query"), stage_id: 1, task_number: 2, }, diff --git a/src/metrics/task_metrics_collector.rs b/src/metrics/task_metrics_collector.rs index e4c054c..bd10d96 100644 --- a/src/metrics/task_metrics_collector.rs +++ b/src/metrics/task_metrics_collector.rs @@ -68,7 +68,7 @@ impl TreeNodeRewriter for TaskMetricsCollector { // sent metrics (the NetworkShuffleExec tracks it for us). Some(_) => { return internal_err!( - "duplicate task metrics for key {} during metrics collection", + "duplicate task metrics for key {:?} during metrics collection", stage_key ); } @@ -127,12 +127,13 @@ mod tests { use datafusion::arrow::record_batch::RecordBatch; use futures::StreamExt; + use crate::DistributedExt; use crate::DistributedPhysicalOptimizerRule; - use crate::metrics::proto::{df_metrics_set_to_proto, metrics_set_proto_to_df}; + use crate::execution_plans::DistributedExec; + use crate::metrics::proto::metrics_set_proto_to_df; use crate::test_utils::in_memory_channel_resolver::InMemoryChannelResolver; use crate::test_utils::plans::{count_plan_nodes, get_stages_and_stage_keys}; use crate::test_utils::session_context::register_temp_parquet_table; - use crate::{DistributedExt, StageExec}; use datafusion::execution::{SessionStateBuilder, context::SessionContext}; use datafusion::prelude::SessionConfig; use datafusion::{ @@ -232,11 +233,14 @@ mod tests { } /// runs a sql query and returns the coordinator StageExec - async fn plan_sql(ctx: &SessionContext, sql: &str) -> StageExec { + async fn plan_sql(ctx: &SessionContext, sql: &str) -> DistributedExec { let df = ctx.sql(sql).await.unwrap(); let physical_distributed = df.create_physical_plan().await.unwrap(); - let stage_exec = match physical_distributed.as_any().downcast_ref::() { + let stage_exec = match physical_distributed + .as_any() + .downcast_ref::() + { Some(stage_exec) => stage_exec.clone(), None => panic!( "expected StageExec from distributed optimization, got: {}", @@ -246,7 +250,7 @@ mod tests { stage_exec } - async fn execute_plan(stage_exec: &StageExec, ctx: &SessionContext) { + async fn execute_plan(stage_exec: &DistributedExec, ctx: &SessionContext) { let task_ctx = ctx.task_ctx(); let stream = stage_exec.execute(0, task_ctx).unwrap(); @@ -279,29 +283,20 @@ mod tests { // Collect metrics for all tasks from the root StageExec. let collector = TaskMetricsCollector::new(); + let result = collector.collect(stage_exec.plan.clone()).unwrap(); - let mut actual_collected_metrics = result.input_task_metrics; - actual_collected_metrics.insert( - StageKey { - query_id: stage_exec.query_id.to_string(), - stage_id: stage_exec.num as u64, - task_number: 0, - }, - result - .task_metrics - .iter() - .map(|m| df_metrics_set_to_proto(m).unwrap()) - .collect::>(), - ); // Ensure that there's metrics for each node for each task for each stage. for expected_stage_key in expected_stage_keys { // Get the collected metrics for this task. - let actual_metrics = actual_collected_metrics.get(&expected_stage_key).unwrap(); + let actual_metrics = result.input_task_metrics.get(&expected_stage_key).unwrap(); // Assert that there's metrics for each node in this task. let stage = stages.get(&(expected_stage_key.stage_id as usize)).unwrap(); - assert_eq!(actual_metrics.len(), count_plan_nodes(&stage.plan)); + assert_eq!( + actual_metrics.len(), + count_plan_nodes(stage.plan.decoded().unwrap()) + ); // Ensure each node has at least one metric which was collected. for metrics_set in actual_metrics.iter() { diff --git a/src/protobuf/app_metadata.rs b/src/protobuf/app_metadata.rs index ee74221..b0de54d 100644 --- a/src/protobuf/app_metadata.rs +++ b/src/protobuf/app_metadata.rs @@ -1,5 +1,5 @@ use crate::metrics::proto::MetricsSetProto; -use crate::protobuf::StageKey; +use crate::protobuf::distributed_codec::StageKey; /// A collection of metrics for a set of tasks in an ExecutionPlan. each /// entry should have a distinct [StageKey]. diff --git a/src/protobuf/distributed_codec.rs b/src/protobuf/distributed_codec.rs index 4107f46..e29422d 100644 --- a/src/protobuf/distributed_codec.rs +++ b/src/protobuf/distributed_codec.rs @@ -1,8 +1,13 @@ use super::get_distributed_user_codecs; +use crate::distributed_physical_optimizer_rule::NetworkBoundary; use crate::execution_plans::{NetworkCoalesceExec, NetworkCoalesceReady, NetworkShuffleReadyExec}; -use crate::{NetworkShuffleExec, PartitionIsolatorExec}; +use crate::stage::MaybeEncodedPlan; +use crate::{ExecutionTask, NetworkShuffleExec, PartitionIsolatorExec, Stage}; +use bytes::Bytes; use datafusion::arrow::datatypes::Schema; use datafusion::arrow::datatypes::SchemaRef; +use datafusion::common::internal_datafusion_err; +use datafusion::error::DataFusionError; use datafusion::execution::FunctionRegistry; use datafusion::physical_expr::EquivalenceProperties; use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; @@ -15,6 +20,7 @@ use datafusion_proto::protobuf; use datafusion_proto::protobuf::proto_error; use prost::Message; use std::sync::Arc; +use url::Url; /// DataFusion [PhysicalExtensionCodec] implementation that allows serializing and /// deserializing the custom ExecutionPlans in this project @@ -50,11 +56,47 @@ impl PhysicalExtensionCodec for DistributedCodec { // user configurations. This is a limitation of the current trait design. let ctx = SessionContext::new(); + fn parse_stage_proto( + proto: Option, + inputs: &[Arc], + ) -> Result { + let Some(proto) = proto else { + return Err(proto_error("Empty StageProto")); + }; + let plan_proto = match proto.plan_proto.is_empty() { + true => None, + false => Some(proto.plan_proto), + }; + + let plan = match (plan_proto, inputs.first()) { + (Some(plan_proto), None) => MaybeEncodedPlan::Encoded(plan_proto), + (None, Some(child)) => MaybeEncodedPlan::Decoded(Arc::clone(child)), + (Some(_), Some(_)) => { + return Err(proto_error( + "When building a Stage from protobuf, either an already decoded child or its serialized bytes must be passed, but not both", + )); + } + (None, None) => { + return Err(proto_error( + "When building a Stage from protobuf, an already decoded child or its serialized bytes must be passed", + )); + } + }; + + Ok(Stage { + query_id: uuid::Uuid::from_slice(proto.query_id.as_ref()) + .map_err(|_| proto_error("Invalid query_id in ExecutionStageProto"))?, + num: proto.num as usize, + plan, + tasks: decode_tasks(proto.tasks)?, + }) + } + match distributed_exec_node { DistributedExecNode::NetworkHashShuffle(NetworkShuffleExecProto { schema, partitioning, - stage_num, + input_stage, }) => { let schema: Schema = schema .as_ref() @@ -72,14 +114,13 @@ impl PhysicalExtensionCodec for DistributedCodec { Ok(Arc::new(new_network_hash_shuffle_exec( partitioning, Arc::new(schema), - stage_num as usize, + parse_stage_proto(input_stage, inputs)?, ))) } DistributedExecNode::NetworkCoalesceTasks(NetworkCoalesceExecProto { schema, partitioning, - stage_num, - input_tasks, + input_stage, }) => { let schema: Schema = schema .as_ref() @@ -97,8 +138,7 @@ impl PhysicalExtensionCodec for DistributedCodec { Ok(Arc::new(new_network_coalesce_tasks_exec( partitioning, Arc::new(schema), - stage_num as usize, - input_tasks as usize, + parse_stage_proto(input_stage, inputs)?, ))) } DistributedExecNode::PartitionIsolator(PartitionIsolatorExecProto { n_tasks }) => { @@ -124,19 +164,29 @@ impl PhysicalExtensionCodec for DistributedCodec { node: Arc, buf: &mut Vec, ) -> datafusion::common::Result<()> { + fn encode_stage_proto(stage: Option<&Stage>) -> Result { + let stage = stage.ok_or(proto_error( + "Cannot encode a NetworkBoundary that has no stage assinged", + ))?; + Ok(StageProto { + query_id: Bytes::from(stage.query_id.as_bytes().to_vec()), + num: stage.num as u64, + tasks: encode_tasks(&stage.tasks), + plan_proto: match &stage.plan { + MaybeEncodedPlan::Decoded(_) => Bytes::new(), + MaybeEncodedPlan::Encoded(proto) => proto.clone(), + }, + }) + } + if let Some(node) = node.as_any().downcast_ref::() { - let NetworkShuffleExec::Ready(ready_node) = node else { - return Err(proto_error( - "deserialized an NetworkShuffleExec that is not ready", - )); - }; let inner = NetworkShuffleExecProto { schema: Some(node.schema().try_into()?), partitioning: Some(serialize_partitioning( node.properties().output_partitioning(), &DistributedCodec {}, )?), - stage_num: ready_node.stage_num as u64, + input_stage: Some(encode_stage_proto(node.input_stage())?), }; let wrapper = DistributedExecProto { @@ -145,20 +195,13 @@ impl PhysicalExtensionCodec for DistributedCodec { wrapper.encode(buf).map_err(|e| proto_error(format!("{e}"))) } else if let Some(node) = node.as_any().downcast_ref::() { - let NetworkCoalesceExec::Ready(ready_node) = node else { - return Err(proto_error( - "deserialized an NetworkCoalesceExec that is not ready", - )); - }; - let inner = NetworkCoalesceExecProto { schema: Some(node.schema().try_into()?), partitioning: Some(serialize_partitioning( node.properties().output_partitioning(), &DistributedCodec {}, )?), - stage_num: ready_node.stage_num as u64, - input_tasks: ready_node.input_tasks as u64, + input_stage: Some(encode_stage_proto(node.input_stage())?), }; let wrapper = DistributedExecProto { @@ -187,9 +230,48 @@ impl PhysicalExtensionCodec for DistributedCodec { } } +/// A key that uniquely identifies a stage in a query +#[derive(Clone, Hash, Eq, PartialEq, ::prost::Message)] +pub struct StageKey { + /// Our query id + #[prost(bytes, tag = "1")] + pub query_id: Bytes, + /// Our stage id + #[prost(uint64, tag = "2")] + pub stage_id: u64, + /// The task number within the stage + #[prost(uint64, tag = "3")] + pub task_number: u64, +} + +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct StageProto { + /// Our query id + #[prost(bytes, tag = "1")] + pub query_id: Bytes, + /// Our stage number + #[prost(uint64, tag = "2")] + pub num: u64, + /// Our tasks which tell us how finely grained to execute the partitions in + /// the plan + #[prost(message, repeated, tag = "3")] + pub tasks: Vec, + /// The child plan already serialized + #[prost(bytes, tag = "4")] + pub plan_proto: Bytes, // Apparently, with an optional keyword, we cannot put Bytes here. +} + +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ExecutionTaskProto { + /// The url of the worker that will execute this task. A None value is interpreted as + /// unassigned. + #[prost(string, optional, tag = "1")] + pub url_str: Option, +} + #[derive(Clone, PartialEq, ::prost::Message)] pub struct DistributedExecProto { - #[prost(oneof = "DistributedExecNode", tags = "1, 2, 3")] + #[prost(oneof = "DistributedExecNode", tags = "1, 2, 3, 4, 5")] pub node: Option, } @@ -218,14 +300,14 @@ pub struct NetworkShuffleExecProto { schema: Option, #[prost(message, optional, tag = "2")] partitioning: Option, - #[prost(uint64, tag = "3")] - stage_num: u64, + #[prost(message, optional, tag = "3")] + input_stage: Option, } fn new_network_hash_shuffle_exec( partitioning: Partitioning, schema: SchemaRef, - stage_num: usize, + input_stage: Stage, ) -> NetworkShuffleExec { NetworkShuffleExec::Ready(NetworkShuffleReadyExec { properties: PlanProperties::new( @@ -234,7 +316,7 @@ fn new_network_hash_shuffle_exec( EmissionType::Incremental, Boundedness::Bounded, ), - stage_num, + input_stage, metrics_collection: Default::default(), }) } @@ -248,17 +330,14 @@ pub struct NetworkCoalesceExecProto { schema: Option, #[prost(message, optional, tag = "2")] partitioning: Option, - #[prost(uint64, tag = "3")] - stage_num: u64, - #[prost(uint64, tag = "4")] - input_tasks: u64, + #[prost(message, optional, tag = "3")] + input_stage: Option, } fn new_network_coalesce_tasks_exec( partitioning: Partitioning, schema: SchemaRef, - stage_num: usize, - input_tasks: usize, + input_stage: Stage, ) -> NetworkCoalesceExec { NetworkCoalesceExec::Ready(NetworkCoalesceReady { properties: PlanProperties::new( @@ -267,23 +346,61 @@ fn new_network_coalesce_tasks_exec( EmissionType::Incremental, Boundedness::Bounded, ), - stage_num, - input_tasks, + input_stage, metrics_collection: Default::default(), }) } +fn encode_tasks(tasks: &[ExecutionTask]) -> Vec { + tasks + .iter() + .map(|task| ExecutionTaskProto { + url_str: task.url.as_ref().map(|v| v.to_string()), + }) + .collect() +} + +fn decode_tasks(tasks: Vec) -> Result, DataFusionError> { + tasks + .into_iter() + .map(|task| { + Ok(ExecutionTask { + url: task + .url_str + .map(|u| { + Url::parse(&u).map_err(|_| internal_datafusion_err!("Invalid URL: {u}")) + }) + .transpose()?, + }) + }) + .collect() +} + #[cfg(test)] mod tests { use super::*; use datafusion::arrow::datatypes::{DataType, Field}; use datafusion::physical_expr::LexOrdering; + use datafusion::physical_plan::empty::EmptyExec; use datafusion::{ execution::registry::MemoryFunctionRegistry, physical_expr::{Partitioning, PhysicalSortExpr, expressions::Column, expressions::col}, physical_plan::{ExecutionPlan, displayable, sorts::sort::SortExec, union::UnionExec}, }; + fn empty_exec() -> Arc { + Arc::new(EmptyExec::new(SchemaRef::new(Schema::empty()))) + } + + fn dummy_stage() -> Stage { + Stage { + query_id: Default::default(), + num: 0, + plan: MaybeEncodedPlan::Decoded(empty_exec()), + tasks: vec![], + } + } + fn schema_i32(name: &str) -> Arc { Arc::new(Schema::new(vec![Field::new(name, DataType::Int32, false)])) } @@ -299,12 +416,13 @@ mod tests { let schema = schema_i32("a"); let part = Partitioning::Hash(vec![Arc::new(Column::new("a", 0))], 4); - let plan: Arc = Arc::new(new_network_hash_shuffle_exec(part, schema, 0)); + let plan: Arc = + Arc::new(new_network_hash_shuffle_exec(part, schema, dummy_stage())); let mut buf = Vec::new(); codec.try_encode(plan.clone(), &mut buf)?; - let decoded = codec.try_decode(&buf, &[], ®istry)?; + let decoded = codec.try_decode(&buf, &[empty_exec()], ®istry)?; assert_eq!(repr(&plan), repr(&decoded)); Ok(()) @@ -319,7 +437,7 @@ mod tests { let flight = Arc::new(new_network_hash_shuffle_exec( Partitioning::UnknownPartitioning(1), schema, - 0, + dummy_stage(), )); let plan: Arc = @@ -343,12 +461,12 @@ mod tests { let left = Arc::new(new_network_hash_shuffle_exec( Partitioning::RoundRobinBatch(2), schema.clone(), - 0, + dummy_stage(), )); let right = Arc::new(new_network_hash_shuffle_exec( Partitioning::RoundRobinBatch(2), schema.clone(), - 1, + dummy_stage(), )); let union = Arc::new(UnionExec::new(vec![left.clone(), right.clone()])); @@ -373,7 +491,7 @@ mod tests { let flight = Arc::new(new_network_hash_shuffle_exec( Partitioning::UnknownPartitioning(1), schema.clone(), - 0, + dummy_stage(), )); let sort_expr = PhysicalSortExpr { diff --git a/src/protobuf/mod.rs b/src/protobuf/mod.rs index 15a8c63..5de1c15 100644 --- a/src/protobuf/mod.rs +++ b/src/protobuf/mod.rs @@ -1,19 +1,15 @@ mod app_metadata; mod distributed_codec; mod errors; -mod stage_proto; mod user_codec; #[allow(unused_imports)] pub(crate) use app_metadata::{AppMetadata, FlightAppMetadata, MetricsCollection, TaskMetrics}; -pub(crate) use distributed_codec::DistributedCodec; +pub(crate) use distributed_codec::{DistributedCodec, StageKey}; pub(crate) use errors::{ datafusion_error_to_tonic_status, map_flight_to_datafusion_error, map_status_to_datafusion_error, }; -#[cfg(test)] -pub(crate) use stage_proto::proto_from_stage; -pub(crate) use stage_proto::{StageKey, proto_from_input_stage, stage_from_proto}; pub(crate) use user_codec::{ get_distributed_user_codecs, set_distributed_user_codec, set_distributed_user_codec_arc, }; diff --git a/src/protobuf/stage_proto.rs b/src/protobuf/stage_proto.rs deleted file mode 100644 index bcf8b74..0000000 --- a/src/protobuf/stage_proto.rs +++ /dev/null @@ -1,301 +0,0 @@ -use crate::execution_plans::{ExecutionTask, InputStage, StageExec}; -use bytes::Bytes; -use datafusion::common::exec_err; -use datafusion::{ - common::internal_datafusion_err, - error::{DataFusionError, Result}, - execution::runtime_env::RuntimeEnv, - prelude::SessionContext, -}; -use datafusion_proto::{ - physical_plan::{AsExecutionPlan, PhysicalExtensionCodec}, - protobuf::PhysicalPlanNode, -}; -use prost::Message; -use std::fmt::Display; -use url::Url; - -/// A key that uniquely identifies a stage in a query -#[derive(Clone, Hash, Eq, PartialEq, ::prost::Message)] -pub struct StageKey { - /// Our query id - #[prost(string, tag = "1")] - pub query_id: String, - /// Our stage id - #[prost(uint64, tag = "2")] - pub stage_id: u64, - /// The task number within the stage - #[prost(uint64, tag = "3")] - pub task_number: u64, -} - -impl Display for StageKey { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!( - f, - "StageKey_QueryID_{}_StageID_{}_TaskNumber_{}", - self.query_id, self.stage_id, self.task_number - ) - } -} - -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct StageExecProto { - /// Our query id - #[prost(bytes, tag = "1")] - query_id: Vec, - /// Our stage number - #[prost(uint64, tag = "2")] - num: u64, - /// Our stage name - #[prost(string, tag = "3")] - name: String, - /// The physical execution plan that this stage will execute. - #[prost(message, optional, boxed, tag = "4")] - plan: Option>, - /// The input stages to this stage - #[prost(repeated, message, tag = "5")] - inputs: Vec, - /// Our tasks which tell us how finely grained to execute the partitions in - /// the plan - #[prost(message, repeated, tag = "6")] - tasks: Vec, -} - -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct StageInputProto { - #[prost(uint64, tag = "1")] - num: u64, - #[prost(message, repeated, tag = "2")] - tasks: Vec, - #[prost(bytes, tag = "3")] - stage: Bytes, -} - -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct ExecutionTaskProto { - /// The url of the worker that will execute this task. A None value is interpreted as - /// unassigned. - #[prost(string, optional, tag = "1")] - url_str: Option, -} - -fn encode_tasks(tasks: &[ExecutionTask]) -> Vec { - tasks - .iter() - .map(|task| ExecutionTaskProto { - url_str: task.url.as_ref().map(|v| v.to_string()), - }) - .collect() -} - -/// Encodes an [InputStage] as protobuf [Bytes]: -/// - If the input is [InputStage::Decoded], it will serialize the inner plan as protobuf bytes. -/// - If the input is [InputStage::Encoded], it will pass through the [Bytes] in a zero-copy manner. -pub(crate) fn proto_from_input_stage( - input_stage: &InputStage, - codec: &dyn PhysicalExtensionCodec, -) -> Result { - match input_stage { - InputStage::Decoded(v) => { - let stage = StageExec::from_dyn(v); - Ok(proto_from_stage(stage, codec)?.encode_to_vec().into()) - } - InputStage::Encoded { proto, .. } => Ok(proto.clone()), - } -} - -/// Converts a [StageExec] into a [StageExecProto], which makes it suitable to be serialized and -/// sent over the wire. -/// -/// If the input [InputStage]s of the provided [StageExec] are already encoded as protobuf [Bytes], -/// they will not be decoded and re-encoded, the [Bytes] are just passthrough as-is in a zero copy -/// manner. -pub(crate) fn proto_from_stage( - stage: &StageExec, - codec: &dyn PhysicalExtensionCodec, -) -> Result { - let proto_plan = PhysicalPlanNode::try_from_physical_plan(stage.plan.clone(), codec)?; - let inputs = stage - .input_stages_iter() - .map(|s| match s { - InputStage::Decoded(s) => { - let Some(s) = s.as_any().downcast_ref::() else { - return exec_err!( - "Programming error: StageExec input must always be other StageExec" - ); - }; - - Ok(StageInputProto { - num: s.num as u64, - tasks: encode_tasks(&s.tasks), - stage: proto_from_stage(s, codec)?.encode_to_vec().into(), - }) - } - InputStage::Encoded { num, tasks, proto } => Ok(StageInputProto { - num: *num as u64, - tasks: encode_tasks(tasks), - stage: proto.clone(), - }), - }) - .collect::>>()?; - - Ok(StageExecProto { - query_id: stage.query_id.as_bytes().to_vec(), - num: stage.num as u64, - name: stage.name(), - plan: Some(Box::new(proto_plan)), - inputs, - tasks: encode_tasks(&stage.tasks), - }) -} - -/// Decodes the provided protobuf [Bytes] as a [StageExec]. Rather than recursively decoding all the -/// input [InputStage]s, it performs a shallow decoding of just the first [StageExec] level, leaving -/// all the inputs in [InputStage::Encoded] state. -/// -/// This prevents decoding and then re-encoding the whole plan recursively, and only decodes the -/// things that are strictly needed. -pub(crate) fn stage_from_proto( - msg: Bytes, - ctx: &SessionContext, - runtime: &RuntimeEnv, - codec: &dyn PhysicalExtensionCodec, -) -> Result { - fn decode_tasks(tasks: Vec) -> Result> { - tasks - .into_iter() - .map(|task| { - Ok(ExecutionTask { - url: task - .url_str - .map(|u| { - Url::parse(&u).map_err(|_| internal_datafusion_err!("Invalid URL: {u}")) - }) - .transpose()?, - }) - }) - .collect() - } - let msg = StageExecProto::decode(msg) - .map_err(|e| internal_datafusion_err!("Cannot decode StageExecProto: {e}"))?; - let plan_node = msg.plan.ok_or(internal_datafusion_err!( - "ExecutionStageMsg is missing the plan" - ))?; - - let plan = plan_node.try_into_physical_plan(ctx, runtime, codec)?; - - let inputs = msg - .inputs - .into_iter() - .map(|s| { - Ok(InputStage::Encoded { - num: s.num as usize, - tasks: decode_tasks(s.tasks)?, - proto: s.stage, - }) - }) - .collect::>>()?; - - Ok(StageExec { - query_id: msg - .query_id - .try_into() - .map_err(|_| internal_datafusion_err!("Invalid query_id in ExecutionStageProto"))?, - num: msg.num as usize, - name: msg.name, - plan, - inputs, - tasks: decode_tasks(msg.tasks)?, - depth: 0, - }) -} - -// add tests for round trip to and from a proto message for ExecutionStage -#[cfg(test)] -mod tests { - use std::sync::Arc; - - use crate::StageExec; - use crate::protobuf::{proto_from_stage, stage_from_proto}; - use datafusion::{ - arrow::{ - array::{RecordBatch, StringArray, UInt8Array}, - datatypes::{DataType, Field, Schema}, - }, - common::internal_datafusion_err, - datasource::MemTable, - error::Result, - execution::context::SessionContext, - }; - use datafusion_proto::physical_plan::DefaultPhysicalExtensionCodec; - use prost::Message; - use uuid::Uuid; - - // create a simple mem table - fn create_mem_table() -> Arc { - let fields = vec![ - Field::new("id", DataType::UInt8, false), - Field::new("data", DataType::Utf8, false), - ]; - let schema = Arc::new(Schema::new(fields)); - - let partitions = RecordBatch::try_new( - schema.clone(), - vec![ - Arc::new(UInt8Array::from(vec![1, 2])), - Arc::new(StringArray::from(vec!["foo", "bar"])), - ], - ) - .unwrap(); - - Arc::new(MemTable::try_new(schema, vec![vec![partitions]]).unwrap()) - } - - #[tokio::test] - #[ignore] - async fn test_execution_stage_proto_round_trip() -> Result<()> { - let ctx = SessionContext::new(); - let mem_table = create_mem_table(); - ctx.register_table("mem_table", mem_table).unwrap(); - - let physical_plan = ctx - .sql("SELECT id, count(*) FROM mem_table group by data") - .await? - .create_physical_plan() - .await?; - - // Wrap it in an ExecutionStage - let stage = StageExec { - query_id: Uuid::new_v4(), - num: 1, - name: "TestStage".to_string(), - plan: physical_plan, - inputs: vec![], - tasks: vec![], - depth: 0, - }; - - // Convert to proto message - let stage_msg = proto_from_stage(&stage, &DefaultPhysicalExtensionCodec {})?; - - // Serialize to bytes - let mut buf = Vec::new(); - stage_msg - .encode(&mut buf) - .map_err(|e| internal_datafusion_err!("couldn't encode {e:#?}"))?; - - // Convert back to ExecutionStage - let round_trip_stage = stage_from_proto( - buf.into(), - &ctx, - ctx.runtime_env().as_ref(), - &DefaultPhysicalExtensionCodec {}, - )?; - - // Compare original and round-tripped stages - assert_eq!(stage.num, round_trip_stage.num); - assert_eq!(stage.name, round_trip_stage.name); - Ok(()) - } -} diff --git a/src/execution_plans/stage.rs b/src/stage.rs similarity index 52% rename from src/execution_plans/stage.rs rename to src/stage.rs index effd075..d1cc3ad 100644 --- a/src/execution_plans/stage.rs +++ b/src/stage.rs @@ -1,15 +1,10 @@ -use crate::channel_resolver_ext::get_distributed_channel_resolver; -use crate::execution_plans::NetworkCoalesceExec; -use crate::{ChannelResolver, NetworkShuffleExec, PartitionIsolatorExec}; -use datafusion::common::{exec_err, internal_datafusion_err, internal_err, plan_err}; -use datafusion::error::{DataFusionError, Result}; +use crate::execution_plans::{DistributedExec, NetworkCoalesceExec}; +use crate::{NetworkShuffleExec, PartitionIsolatorExec}; +use datafusion::common::plan_err; +use datafusion::error::Result; use datafusion::execution::TaskContext; -use datafusion::physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, displayable, -}; -use datafusion::prelude::SessionContext; -use itertools::Itertools; -use rand::Rng; +use datafusion::physical_plan::{ExecutionPlan, ExecutionPlanProperties, displayable}; +use itertools::{Either, Itertools}; use std::collections::VecDeque; use std::sync::Arc; use url::Url; @@ -75,278 +70,109 @@ use uuid::Uuid; /// Stage can complete on its own; its likely holding a leaf node in the overall phyysical plan and /// producing data from a [`DataSourceExec`]. #[derive(Debug, Clone)] -pub struct StageExec { +pub struct Stage { /// Our query_id pub query_id: Uuid, /// Our stage number pub num: usize, - /// Our stage name - pub name: String, /// The physical execution plan that this stage will execute. - pub plan: Arc, - /// The input stages to this stage - pub inputs: Vec, + pub plan: MaybeEncodedPlan, /// Our tasks which tell us how finely grained to execute the partitions in /// the plan pub tasks: Vec, - /// tree depth of our location in the stage tree, used for display only - pub depth: usize, } -/// A [StageExec] that is the input of another [StageExec]. -/// -/// It can be either: -/// - Decoded: the inner [StageExec] is stored as-is. -/// - Encoded: the inner [StageExec] is stored as protobuf [Bytes]. Storing it this way allow us +#[derive(Debug, Clone)] +pub struct ExecutionTask { + /// The url of the worker that will execute this task. A None value is interpreted as + /// unassigned. + pub url: Option, +} + +/// An [ExecutionPlan] that can be either: +/// - Decoded: the inner [ExecutionPlan] is stored as-is. +/// - Encoded: the inner [ExecutionPlan] is stored as protobuf [Bytes]. Storing it this way allow us /// to thread it through the project and eventually send it through gRPC in a zero copy manner. #[derive(Debug, Clone)] -pub enum InputStage { - /// The decoded [StageExec]. Unfortunately, this cannot be an `Arc`, because at - /// some point we need to upcast `&Arc` to `&Arc`, and Rust - /// compiler does not allow it. - /// - /// This is very annoying because it forces us to store it like an `Arc` - /// here even though we know this can only be `Arc`. For this reason - /// [StageExec::from_dyn] was introduced for casting it back to [StageExec]. +pub enum MaybeEncodedPlan { + /// The decoded [ExecutionPlan]. Decoded(Arc), - /// A protobuf encoded version of the [InputStage]. The inner [Bytes] represent the full - /// input [StageExec] encoded in protobuf format. + /// A protobuf encoded version of the [ExecutionPlan]. The inner [Bytes] represent the full + /// input [ExecutionPlan] encoded in protobuf format. /// /// By keeping it encoded, we avoid encoding/decoding it unnecessarily in parts of the project - /// that do not need it. Only the Stage num and the [ExecutionTask]s are left decoded, - /// as typically those are the only things needed by the network boundaries. The [Bytes] can be - /// just passed through in a zero copy manner. - Encoded { - num: usize, - tasks: Vec, - proto: Bytes, - }, + /// that only need it to be decoded. + Encoded(Bytes), } -impl InputStage { - pub fn num(&self) -> usize { +impl MaybeEncodedPlan { + pub fn to_encoded(&self, codec: &dyn PhysicalExtensionCodec) -> Result { + Ok(match self { + Self::Decoded(plan) => Self::Encoded( + PhysicalPlanNode::try_from_physical_plan(Arc::clone(plan), codec)? + .encode_to_vec() + .into(), + ), + Self::Encoded(plan) => Self::Encoded(plan.clone()), + }) + } + + pub fn decoded(&self) -> Result<&Arc> { match self { - InputStage::Decoded(v) => StageExec::from_dyn(v).num, - InputStage::Encoded { num, .. } => *num, + MaybeEncodedPlan::Decoded(v) => Ok(v), + MaybeEncodedPlan::Encoded(_) => plan_err!("Expected plan to be in a decoded state"), } } - pub fn tasks(&self) -> &[ExecutionTask] { + pub fn encoded(&self) -> Result<&Bytes> { match self { - InputStage::Decoded(v) => &StageExec::from_dyn(v).tasks, - InputStage::Encoded { tasks, .. } => tasks, + MaybeEncodedPlan::Decoded(_) => plan_err!("Expected plan to be in a encoded state"), + MaybeEncodedPlan::Encoded(v) => Ok(v), } } } #[derive(Debug, Clone)] -pub struct ExecutionTask { - /// The url of the worker that will execute this task. A None value is interpreted as - /// unassigned. - pub url: Option, -} - -#[derive(Debug, Clone, Default)] pub struct DistributedTaskContext { pub task_index: usize, + pub task_count: usize, } impl DistributedTaskContext { - pub fn new(task_index: usize) -> Self { - Self { task_index } - } - pub fn from_ctx(ctx: &Arc) -> Arc { ctx.session_config() .get_extension::() - .unwrap_or_default() + .unwrap_or(Arc::new(DistributedTaskContext { + task_index: 0, + task_count: 1, + })) } } -impl StageExec { - /// Dangerous way of accessing a [StageExec] out of an `&Arc`. - /// See [InputStage::Decoded] docs for more details about why panicking here is preferred. - pub(crate) fn from_dyn(plan: &Arc) -> &Self { - plan.as_any() - .downcast_ref() - .expect("Programming Error: expected Arc to be of type StageExec") - } - +impl Stage { /// Creates a new `ExecutionStage` with the given plan and inputs. One task will be created /// responsible for partitions in the plan. pub(crate) fn new( query_id: Uuid, num: usize, plan: Arc, - inputs: Vec, n_tasks: usize, ) -> Self { - StageExec { - name: format!("Stage {:<3}", num), + Self { query_id, num, - plan, - inputs: inputs - .into_iter() - .map(|s| InputStage::Decoded(Arc::new(s))) - .collect(), + plan: MaybeEncodedPlan::Decoded(plan), tasks: vec![ExecutionTask { url: None }; n_tasks], - depth: 0, } } - - /// Returns the name of this stage - pub fn name(&self) -> String { - format!("Stage {:<3}", self.num) - } - - /// Returns an iterator over the input stages of this stage cast as &ExecutionStage - /// which can be useful - pub fn input_stages_iter(&self) -> impl Iterator { - self.inputs.iter() - } - - fn try_assign_urls(&self, urls: &[Url]) -> Result { - let assigned_input_stages = self - .input_stages_iter() - .map(|input_stage| { - let InputStage::Decoded(input_stage) = input_stage else { - return exec_err!("Cannot assign URLs to the tasks in an encoded stage"); - }; - StageExec::from_dyn(input_stage).try_assign_urls(urls) - }) - .map_ok(|v| InputStage::Decoded(Arc::new(v))) - .collect::>>()?; - - // pick a random starting position - let mut rng = rand::thread_rng(); - let start_idx = rng.gen_range(0..urls.len()); - - let assigned_tasks = self - .tasks - .iter() - .enumerate() - .map(|(i, _)| ExecutionTask { - url: Some(urls[(start_idx + i) % urls.len()].clone()), - }) - .collect::>(); - - let assigned_stage = StageExec { - query_id: self.query_id, - num: self.num, - name: self.name.clone(), - plan: self.plan.clone(), - inputs: assigned_input_stages, - tasks: assigned_tasks, - depth: self.depth, - }; - - Ok(assigned_stage) - } - - pub fn from_ctx(ctx: &Arc) -> Result, DataFusionError> { - ctx.session_config() - .get_extension::() - .ok_or(internal_datafusion_err!( - "missing ExecutionStage in session config" - )) - } - - pub fn input_stage(&self, stage_num: usize) -> Result<&InputStage, DataFusionError> { - for input_stage in self.input_stages_iter() { - match input_stage { - InputStage::Decoded(v) => { - if StageExec::from_dyn(v).num == stage_num { - return Ok(input_stage); - }; - } - InputStage::Encoded { num, .. } => { - if *num == stage_num { - return Ok(input_stage); - } - } - } - } - internal_err!("no child stage with num {stage_num}") - } -} - -impl ExecutionPlan for StageExec { - fn name(&self) -> &str { - &self.name - } - - fn as_any(&self) -> &dyn std::any::Any { - self - } - - fn children(&self) -> Vec<&Arc> { - self.inputs - .iter() - .filter_map(|v| match v { - InputStage::Decoded(v) => Some(v), - InputStage::Encoded { .. } => None, - }) - .collect() - } - - fn with_new_children( - self: Arc, - _children: Vec>, - ) -> Result> { - plan_err!("with_new_children() not supported for StageExec") - } - - fn properties(&self) -> &datafusion::physical_plan::PlanProperties { - self.plan.properties() - } - - /// Executes a query in a distributed manner. This method will lazily perform URL assignation - /// to all the tasks, therefore, it must only be called once. - /// - /// [StageExec::execute] is only used for starting the distributed query in the same machine - /// that planned it, but it's not used for task execution in `ArrowFlightEndpoint`, there, - /// the inner `stage.plan` is executed directly. - fn execute( - &self, - partition: usize, - context: Arc, - ) -> Result { - if partition > 0 { - // The StageExec node calls try_assign_urls() lazily upon calling .execute(). This means - // that .execute() must only be called once, as we cannot afford to perform several - // random URL assignation while calling multiple partitions, as they will differ, - // producing an invalid plan - return exec_err!( - "an executable StageExec must only have 1 partition, but it was called with partition index {partition}" - ); - } - - let channel_resolver = get_distributed_channel_resolver(context.session_config())?; - - let assigned_stage = self - .try_assign_urls(&channel_resolver.get_urls()?) - .map(Arc::new) - .map_err(|e| DataFusionError::Execution(e.to_string()))?; - - // insert the stage into the context so that ExecutionPlan nodes - // that care about the stage can access it - let config = context - .session_config() - .clone() - .with_extension(assigned_stage.clone()) - .with_extension(Arc::new(DistributedTaskContext { task_index: 0 })); - - let new_ctx = - SessionContext::new_with_config_rt(config, context.runtime_env().clone()).task_ctx(); - - assigned_stage.plan.execute(partition, new_ctx) - } } +use crate::distributed_physical_optimizer_rule::{NetworkBoundary, NetworkBoundaryExt}; use bytes::Bytes; -use datafusion::common::tree_node::{TreeNode, TreeNodeRecursion}; use datafusion::physical_expr::Partitioning; +use datafusion_proto::physical_plan::{AsExecutionPlan, PhysicalExtensionCodec}; +use datafusion_proto::protobuf::PhysicalPlanNode; +use prost::Message; /// Be able to display a nice tree for stages. /// /// The challenge to doing this at the moment is that `TreeRenderVistor` @@ -367,103 +193,93 @@ const LDCORNER: &str = "└"; // Left bottom corner const VERTICAL: &str = "│"; // Vertical line const HORIZONTAL: &str = "─"; // Horizontal line -impl StageExec { - fn format(&self, plan: &dyn ExecutionPlan, indent: usize, f: &mut String) -> std::fmt::Result { - let mut node_str = displayable(plan).one_line().to_string(); - node_str.pop(); - write!(f, "{} {node_str}", " ".repeat(indent))?; - - if let Some(NetworkShuffleExec::Ready(ready)) = - plan.as_any().downcast_ref::() - { - let Ok(input_stage) = &self.input_stage(ready.stage_num) else { - writeln!(f, "Wrong partition number {}", ready.stage_num)?; - return Ok(()); - }; - let n_tasks = self.tasks.len(); - let input_tasks = input_stage.tasks().len(); - let partitions = plan.output_partitioning().partition_count(); - let stage = ready.stage_num; - write!( - f, - " read_from=Stage {stage}, output_partitions={partitions}, n_tasks={n_tasks}, input_tasks={input_tasks}", - )?; - } +pub fn display_plan_ascii(plan: &dyn ExecutionPlan) -> String { + if let Some(plan) = plan.as_any().downcast_ref::() { + let mut f = String::new(); + display_ascii(Either::Left(plan), 0, &mut f).unwrap(); + f + } else { + displayable(plan).indent(true).to_string() + } +} - if let Some(NetworkCoalesceExec::Ready(ready)) = - plan.as_any().downcast_ref::() - { - let Ok(input_stage) = &self.input_stage(ready.stage_num) else { - writeln!(f, "Wrong partition number {}", ready.stage_num)?; - return Ok(()); +fn display_ascii( + stage: Either<&DistributedExec, &Stage>, + depth: usize, + f: &mut String, +) -> std::fmt::Result { + let plan = match stage { + Either::Left(distributed_exec) => distributed_exec.children().first().unwrap(), + Either::Right(stage) => { + let MaybeEncodedPlan::Decoded(plan) = &stage.plan else { + return write!(f, "StageExec: encoded input plan"); }; - let tasks = input_stage.tasks().len(); - let partitions = plan.output_partitioning().partition_count(); - let stage = ready.stage_num; - write!( + plan + } + }; + match stage { + Either::Left(_) => { + writeln!( f, - " read_from=Stage {stage}, output_partitions={partitions}, input_tasks={tasks}", + "{}{}{} DistributedExec {} {}", + " ".repeat(depth), + LTCORNER, + HORIZONTAL.repeat(5), + HORIZONTAL.repeat(2), + format_tasks_for_stage(1, plan) )?; } - - if let Some(isolator) = plan.as_any().downcast_ref::() { - write!( + Either::Right(stage) => { + writeln!( f, - " {}", - format_tasks_for_partition_isolator(isolator, &self.tasks) + "{}{}{} Stage {} {} {}", + " ".repeat(depth), + LTCORNER, + HORIZONTAL.repeat(5), + stage.num, + HORIZONTAL.repeat(2), + format_tasks_for_stage(stage.tasks.len(), plan) )?; } - writeln!(f)?; + } - for child in plan.children() { - self.format(child.as_ref(), indent + 2, f)?; - } - Ok(()) + let mut plan_str = String::new(); + display_inner_ascii(plan, 0, &mut plan_str)?; + let plan_str = plan_str + .split('\n') + .filter(|v| !v.is_empty()) + .collect::>() + .join(&format!("\n{}{}", " ".repeat(depth), VERTICAL)); + writeln!(f, "{}{}{}", " ".repeat(depth), VERTICAL, plan_str)?; + writeln!( + f, + "{}{}{}", + " ".repeat(depth), + LDCORNER, + HORIZONTAL.repeat(50) + )?; + for input_stage in find_input_stages(plan.as_ref()) { + display_ascii(Either::Right(input_stage), depth + 1, f)?; } + Ok(()) } -impl DisplayAs for StageExec { - fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result { - #[allow(clippy::format_in_format_args)] - match t { - DisplayFormatType::Default => { - write!(f, "{}", self.name) - } - DisplayFormatType::Verbose => { - writeln!( - f, - "{}{} {} {}", - LTCORNER, - HORIZONTAL.repeat(5), - self.name, - format_tasks_for_stage(self.tasks.len(), &self.plan) - )?; - - let mut plan_str = String::new(); - self.format(self.plan.as_ref(), 0, &mut plan_str)?; - let plan_str = plan_str - .split('\n') - .filter(|v| !v.is_empty()) - .collect::>() - .join(&format!("\n{}{}", " ".repeat(self.depth), VERTICAL)); - writeln!(f, "{}{}{}", " ".repeat(self.depth), VERTICAL, plan_str)?; - write!( - f, - "{}{}{}", - " ".repeat(self.depth), - LDCORNER, - HORIZONTAL.repeat(50) - )?; +fn display_inner_ascii( + plan: &Arc, + indent: usize, + f: &mut String, +) -> std::fmt::Result { + let node_str = displayable(plan.as_ref()).one_line().to_string(); + writeln!(f, "{} {node_str}", " ".repeat(indent))?; + + if plan.is_network_boundary() { + return Ok(()); + } - Ok(()) - } - DisplayFormatType::TreeRender => write!( - f, - "{}", - format_tasks_for_stage(self.tasks.len(), &self.plan) - ), - } + for child in plan.children() { + display_inner_ascii(child, indent + 2, f)?; } + Ok(()) } fn format_tasks_for_stage(n_tasks: usize, head: &Arc) -> String { @@ -483,29 +299,6 @@ fn format_tasks_for_stage(n_tasks: usize, head: &Arc) -> Stri result } -fn format_tasks_for_partition_isolator( - isolator: &PartitionIsolatorExec, - tasks: &[ExecutionTask], -) -> String { - let input_partitions = isolator.input().output_partitioning().partition_count(); - let partition_groups = PartitionIsolatorExec::partition_groups(input_partitions, tasks.len()); - - let n: usize = partition_groups.iter().map(|v| v.len()).sum(); - let mut partitions = vec![]; - for _ in 0..tasks.len() { - partitions.push(vec!["__".to_string(); n]); - } - - let mut result = "Tasks: ".to_string(); - for (i, partition_group) in partition_groups.iter().enumerate() { - for (j, p) in partition_group.iter().enumerate() { - partitions[i][*p] = format!("p{j}") - } - result += &format!("t{i}:[{}] ", partitions[i].join(",")); - } - result -} - // num_colors must agree with the colorscheme selected from // https://graphviz.org/doc/info/colors.html const NUM_COLORS: usize = 6; @@ -530,34 +323,30 @@ pub fn display_plan_graphviz(plan: Arc) -> Result { COLOR_SCHEME )?; - if plan.as_any().downcast_ref::().is_some() { - // draw all tasks first - plan.apply(|node| { - let stage = node - .as_any() - .downcast_ref::() - .expect("Expected StageExec"); + if plan.as_any().is::() { + let mut max_num = 0; + let mut all_stages = find_all_stages(&plan) + .into_iter() + .inspect(|v| max_num = max_num.max(v.num)) + .collect::>(); + let head_stage = Stage { + query_id: Default::default(), + num: max_num + 1, + plan: MaybeEncodedPlan::Decoded(plan.clone()), + tasks: vec![ExecutionTask { url: None }], + }; + all_stages.insert(0, &head_stage); + // draw all tasks first + for stage in &all_stages { for i in 0..stage.tasks.iter().len() { let p = display_single_task(stage, i)?; writeln!(f, "{}", p)?; } - Ok(TreeNodeRecursion::Continue) - })?; - + } // now draw edges between the tasks - - plan.apply(|node| { - let stage = node - .as_any() - .downcast_ref::() - .expect("Expected StageExec"); - - for input_stage in stage.input_stages_iter() { - let InputStage::Decoded(input_stage) = input_stage else { - continue; - }; - let input_stage = StageExec::from_dyn(input_stage); + for stage in &all_stages { + for input_stage in find_input_stages(stage.plan.decoded()?.as_ref()) { for task_i in 0..stage.tasks.len() { for input_task_i in 0..input_stage.tasks.len() { let edges = @@ -570,9 +359,7 @@ pub fn display_plan_graphviz(plan: Arc) -> Result { } } } - - Ok(TreeNodeRecursion::Continue) - })?; + } } else { // single plan, not a stage tree writeln!(f, "node[shape=none]")?; @@ -585,9 +372,10 @@ pub fn display_plan_graphviz(plan: Arc) -> Result { Ok(f) } -pub fn display_single_task(stage: &StageExec, task_i: usize) -> Result { +fn display_single_task(stage: &Stage, task_i: usize) -> Result { + let plan = stage.plan.decoded()?; let partition_group = - build_partition_group(task_i, stage.plan.output_partitioning().partition_count()); + build_partition_group(task_i, plan.output_partitioning().partition_count()); let mut f = String::new(); writeln!( @@ -618,7 +406,7 @@ pub fn display_single_task(stage: &StageExec, task_i: usize) -> Result { writeln!( f, "{}", - display_plan(&stage.plan, task_i, stage.tasks.len(), stage.num)? + display_plan(plan, task_i, stage.tasks.len(), stage.num)? )?; writeln!(f, " }}")?; writeln!(f, " }}")?; @@ -626,7 +414,7 @@ pub fn display_single_task(stage: &StageExec, task_i: usize) -> Result { Ok(f) } -pub fn display_plan( +fn display_plan( plan: &Arc, task_i: usize, n_tasks: usize, @@ -644,6 +432,10 @@ pub fn display_plan( node_index += 1; let p = display_single_plan(plan.as_ref(), stage_num, task_i, node_index)?; writeln!(f, "{}", p)?; + + if plan.is_network_boundary() { + continue; + } for child in plan.children().iter() { queue.push_back(child); } @@ -701,7 +493,11 @@ pub fn display_plan( } } - for child in plan.children().iter() { + if plan.as_ref().is_network_boundary() { + continue; + } + + for child in plan.children() { queue.push_back((child, Some(plan), node_index)); } } @@ -752,18 +548,17 @@ pub fn display_plan( /// >]; /// ``` pub fn display_single_plan( - plan: &dyn ExecutionPlan, + plan: &(dyn ExecutionPlan + 'static), stage_num: usize, task_i: usize, node_index: usize, ) -> Result { let mut f = String::new(); let output_partitions = plan.output_partitioning().partition_count(); - let input_partitions = if let Some(child) = plan.children().first() { - child.output_partitioning().partition_count() - } else if plan.as_any().is::() || plan.as_any().is::() - { + let input_partitions = if plan.is_network_boundary() { output_partitions + } else if let Some(child) = plan.children().first() { + child.output_partitioning().partition_count() } else { 1 }; @@ -827,41 +622,34 @@ pub fn display_single_plan( } fn display_inter_task_edges( - stage: &StageExec, + stage: &Stage, task_i: usize, - input_stage: &StageExec, + input_stage: &Stage, input_task_i: usize, ) -> Result { + let MaybeEncodedPlan::Decoded(plan) = &stage.plan else { + return plan_err!("The inner plan of a stage was encoded."); + }; + let MaybeEncodedPlan::Decoded(input_plan) = &input_stage.plan else { + return plan_err!("The inner plan of a stage was encoded."); + }; let mut f = String::new(); - let partition_group = - build_partition_group(task_i, stage.plan.output_partitioning().partition_count()); - let mut found_isolator = false; - let mut queue = VecDeque::from([&stage.plan]); + let mut queue = VecDeque::from([plan]); let mut index = 0; while let Some(plan) = queue.pop_front() { index += 1; - if plan.as_any().is::() { - found_isolator = true; - } else if let Some(node) = plan.as_any().downcast_ref::() { - let NetworkShuffleExec::Ready(node) = node else { - continue; - }; - if node.stage_num != input_stage.num { + if let Some(node) = plan.as_any().downcast_ref::() { + if node.input_stage().is_none_or(|v| v.num != input_stage.num) { continue; } // draw the edges to this node pulling data up from its child let output_partitions = plan.output_partitioning().partition_count(); for p in 0..output_partitions { - let mut style = ""; - if found_isolator && !partition_group.contains(&p) { - style = "[style=invis]"; - } - writeln!( f, - " {}_{}_{}_{}:t{}:n -> {}_{}_{}_{}:b{}:s {} [color={}]", - input_stage.plan.name(), + " {}_{}_{}_{}:t{}:n -> {}_{}_{}_{}:b{}:s [color={}]", + input_plan.name(), input_stage.num, input_task_i, 1, // the repartition exec is always the first node in the plan @@ -871,30 +659,22 @@ fn display_inter_task_edges( task_i, index, p, - style, p % NUM_COLORS + 1 )?; } + continue; } else if let Some(node) = plan.as_any().downcast_ref::() { - let NetworkCoalesceExec::Ready(node) = node else { - continue; - }; - if node.stage_num != input_stage.num { + if node.input_stage().is_none_or(|v| v.num != input_stage.num) { continue; } // draw the edges to this node pulling data up from its child let output_partitions = plan.output_partitioning().partition_count(); let input_partitions_per_task = output_partitions / input_stage.tasks.len(); for p in 0..input_partitions_per_task { - let mut style = ""; - if found_isolator && !partition_group.contains(&p) { - style = "[style=invis]"; - } - writeln!( f, - " {}_{}_{}_{}:t{}:n -> {}_{}_{}_{}:b{}:s {} [color={}]", - input_stage.plan.name(), + " {}_{}_{}_{}:t{}:n -> {}_{}_{}_{}:b{}:s [color={}]", + input_plan.name(), input_stage.num, input_task_i, 1, // the repartition exec is always the first node in the plan @@ -904,12 +684,13 @@ fn display_inter_task_edges( task_i, index, p + (input_task_i * input_partitions_per_task), - style, p % NUM_COLORS + 1 )?; } + continue; } - for child in plan.children().iter() { + + for child in plan.children() { queue.push_back(child); } } @@ -928,3 +709,30 @@ fn format_pg(partition_group: &[usize]) -> String { fn build_partition_group(task_i: usize, partitions: usize) -> Vec { ((task_i * partitions)..((task_i + 1) * partitions)).collect::>() } + +fn find_input_stages(plan: &dyn ExecutionPlan) -> Vec<&Stage> { + let mut result = vec![]; + for child in plan.children() { + if let Some(plan) = child.as_network_boundary() { + if let Some(stage) = plan.input_stage() { + result.push(stage); + } + } else { + result.extend(find_input_stages(child.as_ref())); + } + } + result +} + +fn find_all_stages(plan: &Arc) -> Vec<&Stage> { + let mut result = vec![]; + if let Some(plan) = plan.as_network_boundary() { + if let Some(stage) = plan.input_stage() { + result.push(stage); + } + } + for child in plan.children() { + result.extend(find_all_stages(child)); + } + result +} diff --git a/src/test_utils/plans.rs b/src/test_utils/plans.rs index a37622b..b2a61e1 100644 --- a/src/test_utils/plans.rs +++ b/src/test_utils/plans.rs @@ -1,15 +1,13 @@ -use std::sync::Arc; - +use bytes::Bytes; use datafusion::{ common::{HashMap, HashSet}, physical_plan::ExecutionPlan, }; +use std::sync::Arc; -use crate::{ - StageExec, - execution_plans::{InputStage, NetworkCoalesceExec, NetworkShuffleExec}, - protobuf::StageKey, -}; +use crate::distributed_physical_optimizer_rule::NetworkBoundaryExt; +use crate::execution_plans::DistributedExec; +use crate::{Stage, protobuf::StageKey}; /// count_plan_nodes counts the number of execution plan nodes in a plan using BFS traversal. /// This does NOT traverse child stages, only the execution plan tree within this stage. @@ -20,10 +18,12 @@ pub fn count_plan_nodes(plan: &Arc) -> usize { while let Some(plan) = queue.pop() { // Skip [NetworkBoundary] nodes from the count. - if !plan.as_any().is::() && !plan.as_any().is::() { - count += 1; + if plan.as_ref().is_network_boundary() { + continue; } + count += 1; + // Add children to the queue for BFS traversal for child in plan.children() { queue.push(child); @@ -36,11 +36,10 @@ pub fn count_plan_nodes(plan: &Arc) -> usize { /// - a map of all stages /// - a set of all the stage keys (one per task) pub fn get_stages_and_stage_keys( - stage: &StageExec, -) -> (HashMap, HashSet) { - let query_id = stage.query_id; + stage: &DistributedExec, +) -> (HashMap, HashSet) { let mut i = 0; - let mut queue = vec![stage]; + let mut queue = find_input_stages(stage); let mut stage_keys = HashSet::new(); let mut stages_map = HashMap::new(); @@ -52,7 +51,7 @@ pub fn get_stages_and_stage_keys( // Add each task. for j in 0..stage.tasks.len() { let stage_key = StageKey { - query_id: query_id.to_string(), + query_id: Bytes::from(stage.query_id.as_bytes().to_vec()), stage_id: stage.num as u64, task_number: j as u64, }; @@ -60,16 +59,21 @@ pub fn get_stages_and_stage_keys( } // Add any child stages - queue.extend( - stage - .input_stages_iter() - .map(|input_stage| match input_stage { - InputStage::Decoded(plan) => StageExec::from_dyn(plan), - InputStage::Encoded { .. } => { - unimplemented!(); - } - }), - ); + queue.extend(find_input_stages(stage.plan.decoded().unwrap().as_ref())); } (stages_map, stage_keys) } + +fn find_input_stages(plan: &dyn ExecutionPlan) -> Vec<&Stage> { + let mut result = vec![]; + for child in plan.children() { + if let Some(plan) = child.as_network_boundary() { + if let Some(stage) = plan.input_stage() { + result.push(stage); + } + } else { + result.extend(find_input_stages(child.as_ref())); + } + } + result +} diff --git a/tests/custom_config_extension.rs b/tests/custom_config_extension.rs index 4caf949..3c77d57 100644 --- a/tests/custom_config_extension.rs +++ b/tests/custom_config_extension.rs @@ -54,7 +54,7 @@ mod tests { } let plan = DistributedPhysicalOptimizerRule::distribute_plan(plan)?; - let stream = execute_stream(Arc::new(plan), ctx.task_ctx())?; + let stream = execute_stream(plan, ctx.task_ctx())?; // It should not fail. stream.try_collect::>().await?; diff --git a/tests/custom_extension_codec.rs b/tests/custom_extension_codec.rs index b07a2cd..f816780 100644 --- a/tests/custom_extension_codec.rs +++ b/tests/custom_extension_codec.rs @@ -25,6 +25,7 @@ mod tests { use datafusion_distributed::test_utils::localhost::start_localhost_context; use datafusion_distributed::{ DistributedExt, DistributedSessionBuilderContext, PartitionIsolatorExec, assert_snapshot, + display_plan_ascii, }; use datafusion_distributed::{DistributedPhysicalOptimizerRule, NetworkShuffleExec}; use datafusion_proto::physical_plan::PhysicalExtensionCodec; @@ -59,18 +60,18 @@ mod tests { let distributed_plan = build_plan(true)?; let distributed_plan = DistributedPhysicalOptimizerRule::distribute_plan(distributed_plan)?; - assert_snapshot!(displayable(&distributed_plan).indent(true).to_string(), @r" - ┌───── Stage 3 Tasks: t0:[p0] + assert_snapshot!(display_plan_ascii(distributed_plan.as_ref()), @r" + ┌───── DistributedExec ── Tasks: t0:[p0] │ SortExec: expr=[numbers@0 DESC NULLS LAST], preserve_partitioning=[false] │ RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=10 - │ NetworkShuffleExec read_from=Stage 2, output_partitions=10, n_tasks=1, input_tasks=10 + │ [Stage 2] => NetworkShuffleExec: output_partitions=10, input_tasks=10 └────────────────────────────────────────────────── - ┌───── Stage 2 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] t1:[p10,p11,p12,p13,p14,p15,p16,p17,p18,p19] t2:[p20,p21,p22,p23,p24,p25,p26,p27,p28,p29] t3:[p30,p31,p32,p33,p34,p35,p36,p37,p38,p39] t4:[p40,p41,p42,p43,p44,p45,p46,p47,p48,p49] t5:[p50,p51,p52,p53,p54,p55,p56,p57,p58,p59] t6:[p60,p61,p62,p63,p64,p65,p66,p67,p68,p69] t7:[p70,p71,p72,p73,p74,p75,p76,p77,p78,p79] t8:[p80,p81,p82,p83,p84,p85,p86,p87,p88,p89] t9:[p90,p91,p92,p93,p94,p95,p96,p97,p98,p99] + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] t1:[p10,p11,p12,p13,p14,p15,p16,p17,p18,p19] t2:[p20,p21,p22,p23,p24,p25,p26,p27,p28,p29] t3:[p30,p31,p32,p33,p34,p35,p36,p37,p38,p39] t4:[p40,p41,p42,p43,p44,p45,p46,p47,p48,p49] t5:[p50,p51,p52,p53,p54,p55,p56,p57,p58,p59] t6:[p60,p61,p62,p63,p64,p65,p66,p67,p68,p69] t7:[p70,p71,p72,p73,p74,p75,p76,p77,p78,p79] t8:[p80,p81,p82,p83,p84,p85,p86,p87,p88,p89] t9:[p90,p91,p92,p93,p94,p95,p96,p97,p98,p99] │ RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 │ SortExec: expr=[numbers@0 DESC NULLS LAST], preserve_partitioning=[false] - │ NetworkShuffleExec read_from=Stage 1, output_partitions=1, n_tasks=10, input_tasks=1 + │ [Stage 1] => NetworkShuffleExec: output_partitions=1, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] + ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] │ RepartitionExec: partitioning=Hash([numbers@0], 10), input_partitions=1 │ FilterExec: numbers@0 > 1 │ Int64ListExec: length=6 @@ -92,7 +93,7 @@ mod tests { +---------+ "); - let stream = execute_stream(Arc::new(distributed_plan), ctx.task_ctx())?; + let stream = execute_stream(distributed_plan, ctx.task_ctx())?; let batches_distributed = stream.try_collect::>().await?; assert_snapshot!(pretty_format_batches(&batches_distributed).unwrap(), @r" diff --git a/tests/distributed_aggregation.rs b/tests/distributed_aggregation.rs index fa0f3b0..b0ea216 100644 --- a/tests/distributed_aggregation.rs +++ b/tests/distributed_aggregation.rs @@ -7,6 +7,7 @@ mod tests { use datafusion_distributed::test_utils::parquet::register_parquet_tables; use datafusion_distributed::{ DefaultSessionBuilder, DistributedPhysicalOptimizerRule, assert_snapshot, + display_plan_ascii, }; use futures::TryStreamExt; use std::error::Error; @@ -27,9 +28,7 @@ mod tests { .with_network_shuffle_tasks(2) .optimize(physical.clone(), &Default::default())?; - let physical_distributed_str = displayable(physical_distributed.as_ref()) - .indent(true) - .to_string(); + let physical_distributed_str = display_plan_ascii(physical_distributed.as_ref()); assert_snapshot!(physical_str, @r" @@ -47,19 +46,19 @@ mod tests { assert_snapshot!(physical_distributed_str, @r" - ┌───── Stage 2 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[count(*)@0 as count(*), RainToday@1 as RainToday] │ SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] │ SortExec: expr=[count(*)@0 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday, count(Int64(1))@1 as count(Int64(1))] │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 1, output_partitions=3, n_tasks=1, input_tasks=2 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1,p2] t1:[p0,p1,p2] + ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2] t1:[p0,p1,p2] │ RepartitionExec: partitioning=Hash([RainToday@0], 3), input_partitions=2 │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__] t1:[__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__] t1:[__,__,p0] │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday], file_type=parquet └────────────────────────────────────────────────── ", @@ -114,9 +113,7 @@ mod tests { .with_network_coalesce_tasks(6) .optimize(physical.clone(), &Default::default())?; - let physical_distributed_str = displayable(physical_distributed.as_ref()) - .indent(true) - .to_string(); + let physical_distributed_str = display_plan_ascii(physical_distributed.as_ref()); assert_snapshot!(physical_str, @r" @@ -131,20 +128,20 @@ mod tests { assert_snapshot!(physical_distributed_str, @r" - ┌───── Stage 3 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ CoalescePartitionsExec - │ NetworkCoalesceExec read_from=Stage 2, output_partitions=18, input_tasks=6 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=18, input_tasks=6 └────────────────────────────────────────────────── - ┌───── Stage 2 Tasks: t0:[p0,p1,p2] t1:[p0,p1,p2] t2:[p0,p1,p2] t3:[p0,p1,p2] t4:[p0,p1,p2] t5:[p0,p1,p2] + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2] t1:[p0,p1,p2] t2:[p0,p1,p2] t3:[p0,p1,p2] t4:[p0,p1,p2] t5:[p0,p1,p2] │ ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday] │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 1, output_partitions=3, n_tasks=6, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] │ RepartitionExec: partitioning=Hash([RainToday@0], 18), input_partitions=1 │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] - │ PartitionIsolatorExec Tasks: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] + │ 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=[RainToday], file_type=parquet └────────────────────────────────────────────────── ", diff --git a/tests/error_propagation.rs b/tests/error_propagation.rs index 95cf0d0..423e882 100644 --- a/tests/error_propagation.rs +++ b/tests/error_propagation.rs @@ -49,7 +49,7 @@ mod tests { )?); } let plan = DistributedPhysicalOptimizerRule::distribute_plan(plan)?; - let stream = execute_stream(Arc::new(plan), ctx.task_ctx())?; + let stream = execute_stream(plan, ctx.task_ctx())?; let Err(err) = stream.try_collect::>().await else { panic!("Should have failed") diff --git a/tests/highly_distributed_query.rs b/tests/highly_distributed_query.rs index 54b595a..c352bed 100644 --- a/tests/highly_distributed_query.rs +++ b/tests/highly_distributed_query.rs @@ -6,7 +6,7 @@ mod tests { use datafusion_distributed::test_utils::parquet::register_parquet_tables; use datafusion_distributed::{ DefaultSessionBuilder, DistributedPhysicalOptimizerRule, NetworkShuffleExec, - assert_snapshot, + assert_snapshot, display_plan_ascii, }; use futures::TryStreamExt; use std::error::Error; @@ -33,10 +33,7 @@ mod tests { let physical_distributed = DistributedPhysicalOptimizerRule::distribute_plan(physical_distributed)?; - let physical_distributed = Arc::new(physical_distributed); - let physical_distributed_str = displayable(physical_distributed.as_ref()) - .indent(true) - .to_string(); + let physical_distributed_str = display_plan_ascii(physical_distributed.as_ref()); assert_snapshot!(physical_str, @"DataSourceExec: file_groups={1 group: [[/testdata/flights-1m.parquet]]}, projection=[FL_DATE, DEP_DELAY, ARR_DELAY, AIR_TIME, DISTANCE, DEP_TIME, ARR_TIME], file_type=parquet", @@ -44,19 +41,20 @@ mod tests { assert_snapshot!(physical_distributed_str, @r" - ┌───── Stage 4 Tasks: t0:[p0,p1,p2,p3,p4] - │ NetworkShuffleExec input_stage=3, input_partitions=5, input_tasks=1 + ┌───── DistributedExec ── Tasks: t0:[p0] + │ CoalescePartitionsExec + │ [Stage 3] => NetworkShuffleExec: output_partitions=5, input_tasks=5 └────────────────────────────────────────────────── - ┌───── Stage 3 Tasks: t0:[p0,p1,p2,p3,p4] + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4] t1:[p5,p6,p7,p8,p9] t2:[p10,p11,p12,p13,p14] t3:[p15,p16,p17,p18,p19] t4:[p20,p21,p22,p23,p24] │ RepartitionExec: partitioning=RoundRobinBatch(5), input_partitions=10 - │ NetworkShuffleExec input_stage=2, input_partitions=10, input_tasks=1 + │ [Stage 2] => NetworkShuffleExec: output_partitions=10, input_tasks=10 └────────────────────────────────────────────────── - ┌───── Stage 2 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] - │ RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - │ NetworkShuffleExec input_stage=1, input_partitions=1, input_tasks=1 + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23,p24,p25,p26,p27,p28,p29,p30,p31,p32,p33,p34,p35,p36,p37,p38,p39,p40,p41,p42,p43,p44,p45,p46,p47,p48,p49] t1:[p50,p51,p52,p53,p54,p55,p56,p57,p58,p59,p60,p61,p62,p63,p64,p65,p66,p67,p68,p69,p70,p71,p72,p73,p74,p75,p76,p77,p78,p79,p80,p81,p82,p83,p84,p85,p86,p87,p88,p89,p90,p91,p92,p93,p94,p95,p96,p97,p98,p99] t2:[p100,p101,p102,p103,p104,p105,p106,p107,p108,p109,p110,p111,p112,p113,p114,p115,p116,p117,p118,p119,p120,p121,p122,p123,p124,p125,p126,p127,p128,p129,p130,p131,p132,p133,p134,p135,p136,p137,p138,p139,p140,p141,p142,p143,p144,p145,p146,p147,p148,p149] t3:[p150,p151,p152,p153,p154,p155,p156,p157,p158,p159,p160,p161,p162,p163,p164,p165,p166,p167,p168,p169,p170,p171,p172,p173,p174,p175,p176,p177,p178,p179,p180,p181,p182,p183,p184,p185,p186,p187,p188,p189,p190,p191,p192,p193,p194,p195,p196,p197,p198,p199] t4:[p200,p201,p202,p203,p204,p205,p206,p207,p208,p209,p210,p211,p212,p213,p214,p215,p216,p217,p218,p219,p220,p221,p222,p223,p224,p225,p226,p227,p228,p229,p230,p231,p232,p233,p234,p235,p236,p237,p238,p239,p240,p241,p242,p243,p244,p245,p246,p247,p248,p249] t5:[p250,p251,p252,p253,p254,p255,p256,p257,p258,p259,p260,p261,p262,p263,p264,p265,p266,p267,p268,p269,p270,p271,p272,p273,p274,p275,p276,p277,p278,p279,p280,p281,p282,p283,p284,p285,p286,p287,p288,p289,p290,p291,p292,p293,p294,p295,p296,p297,p298,p299] t6:[p300,p301,p302,p303,p304,p305,p306,p307,p308,p309,p310,p311,p312,p313,p314,p315,p316,p317,p318,p319,p320,p321,p322,p323,p324,p325,p326,p327,p328,p329,p330,p331,p332,p333,p334,p335,p336,p337,p338,p339,p340,p341,p342,p343,p344,p345,p346,p347,p348,p349] t7:[p350,p351,p352,p353,p354,p355,p356,p357,p358,p359,p360,p361,p362,p363,p364,p365,p366,p367,p368,p369,p370,p371,p372,p373,p374,p375,p376,p377,p378,p379,p380,p381,p382,p383,p384,p385,p386,p387,p388,p389,p390,p391,p392,p393,p394,p395,p396,p397,p398,p399] t8:[p400,p401,p402,p403,p404,p405,p406,p407,p408,p409,p410,p411,p412,p413,p414,p415,p416,p417,p418,p419,p420,p421,p422,p423,p424,p425,p426,p427,p428,p429,p430,p431,p432,p433,p434,p435,p436,p437,p438,p439,p440,p441,p442,p443,p444,p445,p446,p447,p448,p449] t9:[p450,p451,p452,p453,p454,p455,p456,p457,p458,p459,p460,p461,p462,p463,p464,p465,p466,p467,p468,p469,p470,p471,p472,p473,p474,p475,p476,p477,p478,p479,p480,p481,p482,p483,p484,p485,p486,p487,p488,p489,p490,p491,p492,p493,p494,p495,p496,p497,p498,p499] + │ RepartitionExec: partitioning=RoundRobinBatch(50), input_partitions=1 + │ [Stage 1] => NetworkShuffleExec: output_partitions=1, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0] - │ RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=1 + ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] + │ RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 │ DataSourceExec: file_groups={1 group: [[/testdata/flights-1m.parquet]]}, projection=[FL_DATE, DEP_DELAY, ARR_DELAY, AIR_TIME, DISTANCE, DEP_TIME, ARR_TIME], file_type=parquet └────────────────────────────────────────────────── ", diff --git a/tests/stateful_execution_plan.rs b/tests/stateful_execution_plan.rs index 3371d46..b9530e9 100644 --- a/tests/stateful_execution_plan.rs +++ b/tests/stateful_execution_plan.rs @@ -21,11 +21,12 @@ mod tests { use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::stream::RecordBatchStreamAdapter; use datafusion::physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, displayable, execute_stream, + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, execute_stream, }; use datafusion_distributed::test_utils::localhost::start_localhost_context; use datafusion_distributed::{ DistributedExt, DistributedSessionBuilderContext, PartitionIsolatorExec, assert_snapshot, + display_plan_ascii, }; use datafusion_distributed::{DistributedPhysicalOptimizerRule, NetworkShuffleExec}; use datafusion_proto::physical_plan::PhysicalExtensionCodec; @@ -63,25 +64,25 @@ mod tests { let distributed_plan = build_plan()?; let distributed_plan = DistributedPhysicalOptimizerRule::distribute_plan(distributed_plan)?; - assert_snapshot!(displayable(&distributed_plan).indent(true).to_string(), @r" - ┌───── Stage 3 Tasks: t0:[p0] + assert_snapshot!(display_plan_ascii(distributed_plan.as_ref()), @r" + ┌───── DistributedExec ── Tasks: t0:[p0] │ SortExec: expr=[numbers@0 DESC NULLS LAST], preserve_partitioning=[false] │ RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=10 - │ NetworkShuffleExec read_from=Stage 2, output_partitions=10, n_tasks=1, input_tasks=10 + │ [Stage 2] => NetworkShuffleExec: output_partitions=10, input_tasks=10 └────────────────────────────────────────────────── - ┌───── Stage 2 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] t1:[p10,p11,p12,p13,p14,p15,p16,p17,p18,p19] t2:[p20,p21,p22,p23,p24,p25,p26,p27,p28,p29] t3:[p30,p31,p32,p33,p34,p35,p36,p37,p38,p39] t4:[p40,p41,p42,p43,p44,p45,p46,p47,p48,p49] t5:[p50,p51,p52,p53,p54,p55,p56,p57,p58,p59] t6:[p60,p61,p62,p63,p64,p65,p66,p67,p68,p69] t7:[p70,p71,p72,p73,p74,p75,p76,p77,p78,p79] t8:[p80,p81,p82,p83,p84,p85,p86,p87,p88,p89] t9:[p90,p91,p92,p93,p94,p95,p96,p97,p98,p99] + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] t1:[p10,p11,p12,p13,p14,p15,p16,p17,p18,p19] t2:[p20,p21,p22,p23,p24,p25,p26,p27,p28,p29] t3:[p30,p31,p32,p33,p34,p35,p36,p37,p38,p39] t4:[p40,p41,p42,p43,p44,p45,p46,p47,p48,p49] t5:[p50,p51,p52,p53,p54,p55,p56,p57,p58,p59] t6:[p60,p61,p62,p63,p64,p65,p66,p67,p68,p69] t7:[p70,p71,p72,p73,p74,p75,p76,p77,p78,p79] t8:[p80,p81,p82,p83,p84,p85,p86,p87,p88,p89] t9:[p90,p91,p92,p93,p94,p95,p96,p97,p98,p99] │ RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 │ SortExec: expr=[numbers@0 DESC NULLS LAST], preserve_partitioning=[false] - │ NetworkShuffleExec read_from=Stage 1, output_partitions=1, n_tasks=10, input_tasks=1 + │ [Stage 1] => NetworkShuffleExec: output_partitions=1, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] + ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] │ RepartitionExec: partitioning=Hash([numbers@0], 10), input_partitions=1 │ FilterExec: numbers@0 > 1 │ StatefulInt64ListExec: length=6 └────────────────────────────────────────────────── "); - let stream = execute_stream(Arc::new(distributed_plan), ctx.task_ctx())?; + let stream = execute_stream(distributed_plan, ctx.task_ctx())?; let batches_distributed = stream.try_collect::>().await?; assert_snapshot!(pretty_format_batches(&batches_distributed).unwrap(), @r" diff --git a/tests/tpch_validation_test.rs b/tests/tpch_validation_test.rs index d31b7d3..273320d 100644 --- a/tests/tpch_validation_test.rs +++ b/tests/tpch_validation_test.rs @@ -2,12 +2,13 @@ mod tests { use datafusion::error::DataFusionError; use datafusion::execution::{SessionState, SessionStateBuilder}; - use datafusion::physical_plan::{displayable, execute_stream}; + use datafusion::physical_plan::execute_stream; use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_distributed::test_utils::localhost::start_localhost_context; use datafusion_distributed::test_utils::tpch; use datafusion_distributed::{ DistributedPhysicalOptimizerRule, DistributedSessionBuilderContext, assert_snapshot, + display_plan_ascii, }; use futures::TryStreamExt; use std::error::Error; @@ -26,24 +27,24 @@ mod tests { async fn test_tpch_1() -> Result<(), Box> { let plan = test_tpch_query(1).await?; assert_snapshot!(plan, @r" - ┌───── Stage 3 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST] - │ NetworkCoalesceExec read_from=Stage 2, output_partitions=24, input_tasks=4 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] │ SortExec: expr=[l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus, sum(lineitem.l_quantity)@2 as sum_qty, sum(lineitem.l_extendedprice)@3 as sum_base_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@4 as sum_disc_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax)@5 as sum_charge, avg(lineitem.l_quantity)@6 as avg_qty, avg(lineitem.l_extendedprice)@7 as avg_price, avg(lineitem.l_discount)@8 as avg_disc, count(Int64(1))@9 as count_order] │ AggregateExec: mode=FinalPartitioned, gby=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(Int64(1))] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 1, output_partitions=6, n_tasks=4, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] │ RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 24), input_partitions=2 │ AggregateExec: mode=Partial, gby=[l_returnflag@5 as l_returnflag, l_linestatus@6 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(Int64(1))] │ ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as __common_expr_1, l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: l_shipdate@6 <= 1998-09-02, projection=[l_quantity@0, l_extendedprice@1, l_discount@2, l_tax@3, l_returnflag@4, l_linestatus@5] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], file_type=parquet, predicate=l_shipdate@6 <= 1998-09-02, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@0 <= 1998-09-02, required_guarantees=[] └────────────────────────────────────────────────── "); @@ -54,26 +55,26 @@ mod tests { async fn test_tpch_2() -> Result<(), Box> { let plan = test_tpch_query(2).await?; assert_snapshot!(plan, @r" - ┌───── Stage 8 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST] - │ NetworkCoalesceExec read_from=Stage 7, output_partitions=24, input_tasks=4 + │ [Stage 7] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 7 Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 7 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] │ SortExec: expr=[s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[s_acctbal@5 as s_acctbal, s_name@2 as s_name, n_name@7 as n_name, p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_address@3 as s_address, s_phone@4 as s_phone, s_comment@6 as s_comment] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@1), (ps_supplycost@7, min(partsupp.ps_supplycost)@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, n_name@8] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=1 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1 │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 6, output_partitions=6, n_tasks=4, input_tasks=3 + │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 3 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] │ RepartitionExec: partitioning=Hash([p_partkey@0, ps_supplycost@7], 24), input_partitions=6 │ CoalesceBatchesExec: target_batch_size=8192 │ 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 - │ NetworkCoalesceExec read_from=Stage 1, output_partitions=8, input_tasks=4 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, 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] │ CoalesceBatchesExec: target_batch_size=8192 │ 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] @@ -87,35 +88,35 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ 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 - │ NetworkCoalesceExec read_from=Stage 2, output_partitions=8, input_tasks=4 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/partsupp/1.parquet:.., /testdata/tpch/data/partsupp/10.parquet:.., /testdata/tpch/data/partsupp/11.parquet:..], [/testdata/tpch/data/partsupp/11.parquet:.., /testdata/tpch/data/partsupp/12.parquet:.., /testdata/tpch/data/partsupp/13.parquet:.., /testdata/tpch/data/partsupp/14.parquet:..], [/testdata/tpch/data/partsupp/14.parquet:.., /testdata/tpch/data/partsupp/15.parquet:.., /testdata/tpch/data/partsupp/16.parquet:.., /testdata/tpch/data/partsupp/2.parquet:..], [/testdata/tpch/data/partsupp/2.parquet:.., /testdata/tpch/data/partsupp/3.parquet:.., /testdata/tpch/data/partsupp/4.parquet:..], [/testdata/tpch/data/partsupp/4.parquet:.., /testdata/tpch/data/partsupp/5.parquet:.., /testdata/tpch/data/partsupp/6.parquet:.., /testdata/tpch/data/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/region/1.parquet, /testdata/tpch/data/region/10.parquet, /testdata/tpch/data/region/11.parquet], [/testdata/tpch/data/region/12.parquet, /testdata/tpch/data/region/13.parquet, /testdata/tpch/data/region/14.parquet], [/testdata/tpch/data/region/15.parquet, /testdata/tpch/data/region/16.parquet, /testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.parquet, /testdata/tpch/data/region/4.parquet, /testdata/tpch/data/region/5.parquet], [/testdata/tpch/data/region/6.parquet, /testdata/tpch/data/region/7.parquet, /testdata/tpch/data/region/8.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,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 2 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@1] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_mfgr, p_type, p_size], file_type=parquet, predicate=p_size@3 = 15 AND p_type@2 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 6 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 6 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] │ RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 24), input_partitions=6 │ ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] │ AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 5, output_partitions=6, n_tasks=3, input_tasks=1 + │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 5 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 5 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] │ RepartitionExec: partitioning=Hash([ps_partkey@0], 18), input_partitions=6 │ AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@2)], projection=[ps_partkey@1, ps_supplycost@2] │ CoalescePartitionsExec - │ NetworkCoalesceExec read_from=Stage 4, output_partitions=8, input_tasks=4 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ ProjectionExec: expr=[ps_partkey@1 as ps_partkey, ps_supplycost@2 as ps_supplycost, n_regionkey@0 as n_regionkey] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_regionkey@1, ps_partkey@2, ps_supplycost@3] @@ -128,10 +129,10 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/supplier/1.parquet, /testdata/tpch/data/supplier/10.parquet, /testdata/tpch/data/supplier/11.parquet], [/testdata/tpch/data/supplier/12.parquet, /testdata/tpch/data/supplier/13.parquet, /testdata/tpch/data/supplier/14.parquet], [/testdata/tpch/data/supplier/15.parquet, /testdata/tpch/data/supplier/16.parquet, /testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet, /testdata/tpch/data/supplier/4.parquet, /testdata/tpch/data/supplier/5.parquet], [/testdata/tpch/data/supplier/6.parquet, /testdata/tpch/data/supplier/7.parquet, /testdata/tpch/data/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/partsupp/1.parquet:.., /testdata/tpch/data/partsupp/10.parquet:.., /testdata/tpch/data/partsupp/11.parquet:..], [/testdata/tpch/data/partsupp/11.parquet:.., /testdata/tpch/data/partsupp/12.parquet:.., /testdata/tpch/data/partsupp/13.parquet:.., /testdata/tpch/data/partsupp/14.parquet:..], [/testdata/tpch/data/partsupp/14.parquet:.., /testdata/tpch/data/partsupp/15.parquet:.., /testdata/tpch/data/partsupp/16.parquet:.., /testdata/tpch/data/partsupp/2.parquet:..], [/testdata/tpch/data/partsupp/2.parquet:.., /testdata/tpch/data/partsupp/3.parquet:.., /testdata/tpch/data/partsupp/4.parquet:..], [/testdata/tpch/data/partsupp/4.parquet:.., /testdata/tpch/data/partsupp/5.parquet:.., /testdata/tpch/data/partsupp/6.parquet:.., /testdata/tpch/data/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── - ┌───── Stage 4 Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 4 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/region/1.parquet, /testdata/tpch/data/region/10.parquet, /testdata/tpch/data/region/11.parquet], [/testdata/tpch/data/region/12.parquet, /testdata/tpch/data/region/13.parquet, /testdata/tpch/data/region/14.parquet], [/testdata/tpch/data/region/15.parquet, /testdata/tpch/data/region/16.parquet, /testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.parquet, /testdata/tpch/data/region/4.parquet, /testdata/tpch/data/region/5.parquet], [/testdata/tpch/data/region/6.parquet, /testdata/tpch/data/region/7.parquet, /testdata/tpch/data/region/8.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)] └────────────────────────────────────────────────── "); @@ -142,18 +143,18 @@ mod tests { async fn test_tpch_3() -> Result<(), Box> { let plan = test_tpch_query(3).await?; assert_snapshot!(plan, @r" - ┌───── Stage 4 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [revenue@1 DESC, o_orderdate@2 ASC NULLS LAST] - │ NetworkCoalesceExec read_from=Stage 3, output_partitions=24, input_tasks=4 + │ [Stage 3] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 3 Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] │ SortExec: expr=[revenue@1 DESC, o_orderdate@2 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[l_orderkey@0 as l_orderkey, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@3 as revenue, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority] │ AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 2, output_partitions=6, n_tasks=4, input_tasks=1 + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 2 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] │ RepartitionExec: partitioning=Hash([l_orderkey@0, o_orderdate@1, o_shippriority@2], 24), input_partitions=6 │ 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)] │ CoalesceBatchesExec: target_batch_size=8192 @@ -162,7 +163,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4] │ CoalescePartitionsExec - │ NetworkCoalesceExec read_from=Stage 1, output_partitions=8, input_tasks=4 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: o_orderdate@2 < 1995-03-15 │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], file_type=parquet, predicate=o_orderdate@2 < 1995-03-15 AND DynamicFilterPhysicalExpr [ true ], pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@0 < 1995-03-15, required_guarantees=[] @@ -170,10 +171,10 @@ mod tests { │ FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 > 1995-03-15 AND DynamicFilterPhysicalExpr [ true ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 > 1995-03-15, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_mktsegment], file_type=parquet, predicate=c_mktsegment@1 = 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)] └────────────────────────────────────────────────── "); @@ -184,32 +185,32 @@ mod tests { async fn test_tpch_4() -> Result<(), Box> { let plan = test_tpch_query(4).await?; assert_snapshot!(plan, @r" - ┌───── Stage 4 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST] - │ NetworkCoalesceExec read_from=Stage 3, output_partitions=24, input_tasks=4 + │ [Stage 3] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 3 Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] │ SortExec: expr=[o_orderpriority@0 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[o_orderpriority@0 as o_orderpriority, count(Int64(1))@1 as order_count] │ AggregateExec: mode=FinalPartitioned, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 2, output_partitions=6, n_tasks=4, input_tasks=1 + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 2 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] │ RepartitionExec: partitioning=Hash([o_orderpriority@0], 24), input_partitions=6 │ AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(l_orderkey@0, o_orderkey@0)], projection=[o_orderpriority@1] │ CoalescePartitionsExec - │ NetworkCoalesceExec read_from=Stage 1, output_partitions=8, input_tasks=4 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderdate, o_orderpriority], file_type=parquet, predicate=o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1993-07-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 < 1993-10-01, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@2 > l_commitdate@1 └────────────────────────────────────────────────── "); @@ -220,24 +221,24 @@ mod tests { async fn test_tpch_5() -> Result<(), Box> { let plan = test_tpch_query(5).await?; assert_snapshot!(plan, @r" - ┌───── Stage 5 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [revenue@1 DESC] - │ NetworkCoalesceExec read_from=Stage 4, output_partitions=24, input_tasks=4 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] │ SortExec: expr=[revenue@1 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[n_name@0 as n_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as revenue] │ AggregateExec: mode=FinalPartitioned, gby=[n_name@0 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=1 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 3 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] │ RepartitionExec: partitioning=Hash([n_name@0], 24), input_partitions=6 │ AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@3)], projection=[l_extendedprice@1, l_discount@2, n_name@3] │ CoalescePartitionsExec - │ NetworkCoalesceExec read_from=Stage 1, output_partitions=8, input_tasks=4 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, 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] │ CoalesceBatchesExec: target_batch_size=8192 │ 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] @@ -255,20 +256,20 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_custkey@1, c_custkey@0)], projection=[o_orderkey@0, c_nationkey@3] │ CoalescePartitionsExec - │ NetworkCoalesceExec read_from=Stage 2, output_partitions=8, input_tasks=4 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/region/1.parquet, /testdata/tpch/data/region/10.parquet, /testdata/tpch/data/region/11.parquet], [/testdata/tpch/data/region/12.parquet, /testdata/tpch/data/region/13.parquet, /testdata/tpch/data/region/14.parquet], [/testdata/tpch/data/region/15.parquet, /testdata/tpch/data/region/16.parquet, /testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.parquet, /testdata/tpch/data/region/4.parquet, /testdata/tpch/data/region/5.parquet], [/testdata/tpch/data/region/6.parquet, /testdata/tpch/data/region/7.parquet, /testdata/tpch/data/region/8.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,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 2 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, 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=[] └────────────────────────────────────────────────── "); @@ -279,17 +280,17 @@ mod tests { async fn test_tpch_6() -> Result<(), Box> { let plan = test_tpch_query(6).await?; assert_snapshot!(plan, @r" - ┌───── Stage 2 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * lineitem.l_discount)@0 as revenue] │ AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] │ CoalescePartitionsExec - │ NetworkCoalesceExec read_from=Stage 1, output_partitions=8, input_tasks=4 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2, projection=[l_extendedprice@1, l_discount@2] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1994-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1995-01-01 AND l_discount_null_count@5 != row_count@2 AND l_discount_max@4 >= Some(5),15,2 AND l_discount_null_count@5 != row_count@2 AND l_discount_min@6 <= Some(7),15,2 AND l_quantity_null_count@8 != row_count@2 AND l_quantity_min@7 < Some(2400),15,2, required_guarantees=[] └────────────────────────────────────────────────── "); @@ -300,65 +301,65 @@ mod tests { async fn test_tpch_7() -> Result<(), Box> { let plan = test_tpch_query(7).await?; assert_snapshot!(plan, @r" - ┌───── Stage 9 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST] - │ NetworkCoalesceExec read_from=Stage 8, output_partitions=24, input_tasks=4 + │ [Stage 8] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 8 Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 8 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] │ SortExec: expr=[supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year, sum(shipping.volume)@3 as revenue] │ AggregateExec: mode=FinalPartitioned, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 7, output_partitions=6, n_tasks=4, input_tasks=1 + │ [Stage 7] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 7 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 7 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] │ RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 24), input_partitions=6 │ 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] │ CoalesceBatchesExec: target_batch_size=8192 │ 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 - │ NetworkCoalesceExec read_from=Stage 1, output_partitions=8, input_tasks=4 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, 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] │ CoalesceBatchesExec: target_batch_size=8192 │ 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 - │ NetworkCoalesceExec read_from=Stage 2, output_partitions=8, input_tasks=4 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, 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] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, 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] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 3, output_partitions=6, n_tasks=1, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3 │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 6, output_partitions=6, n_tasks=1, input_tasks=3 + │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.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,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 2 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.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,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] │ RepartitionExec: partitioning=Hash([c_custkey@0], 6), input_partitions=2 - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── - ┌───── Stage 6 Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 6 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] │ RepartitionExec: partitioning=Hash([o_custkey@4], 6), input_partitions=6 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@1, o_orderkey@0)], projection=[s_nationkey@0, l_extendedprice@2, l_discount@3, l_shipdate@4, o_custkey@6] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 4, output_partitions=6, n_tasks=3, input_tasks=1 + │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=1 │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 5, output_partitions=6, n_tasks=3, input_tasks=3 + │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 4 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] │ RepartitionExec: partitioning=Hash([l_orderkey@1], 18), input_partitions=6 │ CoalesceBatchesExec: target_batch_size=8192 │ 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] @@ -368,9 +369,9 @@ mod tests { │ FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 AND DynamicFilterPhysicalExpr [ true ], 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 5 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 5 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] │ RepartitionExec: partitioning=Hash([o_orderkey@0], 18), input_partitions=2 - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── "); @@ -381,25 +382,25 @@ mod tests { async fn test_tpch_8() -> Result<(), Box> { let plan = test_tpch_query(8).await?; assert_snapshot!(plan, @r#" - ┌───── Stage 9 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [o_year@0 ASC NULLS LAST] - │ NetworkCoalesceExec read_from=Stage 8, output_partitions=24, input_tasks=4 + │ [Stage 8] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 8 Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 8 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] │ SortExec: expr=[o_year@0 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[o_year@0 as o_year, sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END)@1 / sum(all_nations.volume)@2 as mkt_share] │ AggregateExec: mode=FinalPartitioned, gby=[o_year@0 as o_year], aggr=[sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 7, output_partitions=6, n_tasks=4, input_tasks=1 + │ [Stage 7] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 7 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 7 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] │ RepartitionExec: partitioning=Hash([o_year@0], 24), input_partitions=6 │ 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] │ CoalesceBatchesExec: target_batch_size=8192 │ 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 - │ NetworkCoalesceExec read_from=Stage 1, output_partitions=8, input_tasks=4 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, 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] │ CoalesceBatchesExec: target_batch_size=8192 │ 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] @@ -413,34 +414,34 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@3, c_custkey@0)], projection=[l_extendedprice@0, l_discount@1, s_nationkey@2, o_orderdate@4, c_nationkey@6] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 5, output_partitions=6, n_tasks=1, input_tasks=3 + │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=3 │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 6, output_partitions=6, n_tasks=1, input_tasks=3 + │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/region/1.parquet, /testdata/tpch/data/region/10.parquet, /testdata/tpch/data/region/11.parquet], [/testdata/tpch/data/region/12.parquet, /testdata/tpch/data/region/13.parquet, /testdata/tpch/data/region/14.parquet], [/testdata/tpch/data/region/15.parquet, /testdata/tpch/data/region/16.parquet, /testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.parquet, /testdata/tpch/data/region/4.parquet, /testdata/tpch/data/region/5.parquet], [/testdata/tpch/data/region/6.parquet, /testdata/tpch/data/region/7.parquet, /testdata/tpch/data/region/8.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 5 Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 5 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] │ RepartitionExec: partitioning=Hash([o_custkey@3], 6), input_partitions=6 │ 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] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, 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] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 2, output_partitions=6, n_tasks=3, input_tasks=3 + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=3 │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 4, output_partitions=6, n_tasks=3, input_tasks=1 + │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 2 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] │ RepartitionExec: partitioning=Hash([o_orderkey@0], 18), input_partitions=2 │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31, 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 4 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] │ RepartitionExec: partitioning=Hash([l_orderkey@0], 18), input_partitions=6 │ 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] │ CoalesceBatchesExec: target_batch_size=8192 @@ -450,18 +451,18 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ 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 - │ NetworkCoalesceExec read_from=Stage 3, output_partitions=8, input_tasks=4 + │ [Stage 3] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── - ┌───── Stage 3 Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 3 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: p_type@1 = ECONOMY ANODIZED STEEL, projection=[p_partkey@0] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_type], file_type=parquet, predicate=p_type@1 = 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)] └────────────────────────────────────────────────── - ┌───── Stage 6 Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 6 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] │ RepartitionExec: partitioning=Hash([c_custkey@0], 6), input_partitions=2 - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── "#); @@ -472,18 +473,18 @@ mod tests { async fn test_tpch_9() -> Result<(), Box> { let plan = test_tpch_query(9).await?; assert_snapshot!(plan, @r" - ┌───── Stage 8 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [nation@0 ASC NULLS LAST, o_year@1 DESC] - │ NetworkCoalesceExec read_from=Stage 7, output_partitions=24, input_tasks=4 + │ [Stage 7] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 7 Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 7 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] │ SortExec: expr=[nation@0 ASC NULLS LAST, o_year@1 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[nation@0 as nation, o_year@1 as o_year, sum(profit.amount)@2 as sum_profit] │ AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 6, output_partitions=6, n_tasks=4, input_tasks=1 + │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 6 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 6 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] │ RepartitionExec: partitioning=Hash([nation@0, o_year@1], 24), input_partitions=6 │ 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] @@ -495,25 +496,25 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, 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] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 1, output_partitions=6, n_tasks=1, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=3 │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 5, output_partitions=6, n_tasks=1, input_tasks=3 + │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] │ RepartitionExec: partitioning=Hash([o_orderkey@0], 6), input_partitions=2 - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderdate], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 5 Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 5 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] │ RepartitionExec: partitioning=Hash([l_orderkey@0], 6), input_partitions=6 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, ps_suppkey@1), (l_partkey@1, ps_partkey@0)], projection=[l_orderkey@0, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@9] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 3, output_partitions=6, n_tasks=3, input_tasks=1 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1 │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 4, output_partitions=6, n_tasks=3, input_tasks=3 + │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 3 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] │ RepartitionExec: partitioning=Hash([l_suppkey@2, l_partkey@1], 18), input_partitions=6 │ 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] │ CoalesceBatchesExec: target_batch_size=8192 @@ -523,18 +524,18 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ 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 - │ NetworkCoalesceExec read_from=Stage 2, output_partitions=8, input_tasks=4 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── - ┌───── Stage 2 Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 2 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: p_name@1 LIKE %green%, projection=[p_partkey@0] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_name], file_type=parquet, predicate=p_name@1 LIKE %green% └────────────────────────────────────────────────── - ┌───── Stage 4 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] │ RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 18), input_partitions=2 - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/partsupp/1.parquet:.., /testdata/tpch/data/partsupp/10.parquet:.., /testdata/tpch/data/partsupp/11.parquet:..], [/testdata/tpch/data/partsupp/11.parquet:.., /testdata/tpch/data/partsupp/12.parquet:.., /testdata/tpch/data/partsupp/13.parquet:.., /testdata/tpch/data/partsupp/14.parquet:..], [/testdata/tpch/data/partsupp/14.parquet:.., /testdata/tpch/data/partsupp/15.parquet:.., /testdata/tpch/data/partsupp/16.parquet:.., /testdata/tpch/data/partsupp/2.parquet:..], [/testdata/tpch/data/partsupp/2.parquet:.., /testdata/tpch/data/partsupp/3.parquet:.., /testdata/tpch/data/partsupp/4.parquet:..], [/testdata/tpch/data/partsupp/4.parquet:.., /testdata/tpch/data/partsupp/5.parquet:.., /testdata/tpch/data/partsupp/6.parquet:.., /testdata/tpch/data/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── "); @@ -545,18 +546,18 @@ mod tests { async fn test_tpch_10() -> Result<(), Box> { let plan = test_tpch_query(10).await?; assert_snapshot!(plan, @r" - ┌───── Stage 4 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [revenue@2 DESC] - │ NetworkCoalesceExec read_from=Stage 3, output_partitions=24, input_tasks=4 + │ [Stage 3] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 3 Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] │ SortExec: expr=[revenue@2 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@7 as revenue, c_acctbal@2 as c_acctbal, n_name@4 as n_name, c_address@5 as c_address, c_phone@3 as c_phone, c_comment@6 as c_comment] │ AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@2 as c_acctbal, c_phone@3 as c_phone, n_name@4 as n_name, c_address@5 as c_address, c_comment@6 as c_comment], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 2, output_partitions=6, n_tasks=4, input_tasks=1 + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 2 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] │ RepartitionExec: partitioning=Hash([c_custkey@0, c_name@1, c_acctbal@2, c_phone@3, n_name@4, c_address@5, c_comment@6], 24), input_partitions=6 │ 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] @@ -571,16 +572,16 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_custkey@1, c_custkey@0)], projection=[o_orderkey@0, c_custkey@2, c_name@3, c_address@4, c_nationkey@5, c_phone@6, c_acctbal@7, c_comment@8] │ CoalescePartitionsExec - │ NetworkCoalesceExec read_from=Stage 1, output_partitions=8, input_tasks=4 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], file_type=parquet, predicate=l_returnflag@3 = R AND DynamicFilterPhysicalExpr [ true ], 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,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, 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=[] └────────────────────────────────────────────────── "); @@ -591,7 +592,7 @@ mod tests { async fn test_tpch_11() -> Result<(), Box> { let plan = test_tpch_query(11).await?; assert_snapshot!(plan, @r" - ┌───── Stage 3 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [value@1 DESC] │ SortExec: expr=[value@1 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[ps_partkey@0 as ps_partkey, sum(partsupp.ps_supplycost * partsupp.ps_availqty)@1 as value] @@ -603,7 +604,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[ps_availqty@1, ps_supplycost@2] │ CoalescePartitionsExec - │ NetworkCoalesceExec read_from=Stage 1, output_partitions=8, input_tasks=4 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ ProjectionExec: expr=[ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost, s_nationkey@0 as s_nationkey] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_nationkey@1, ps_availqty@3, ps_supplycost@4] @@ -617,7 +618,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[ps_partkey@1, ps_availqty@2, ps_supplycost@3] │ CoalescePartitionsExec - │ NetworkCoalesceExec read_from=Stage 2, output_partitions=8, input_tasks=4 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, 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] │ CoalesceBatchesExec: target_batch_size=8192 │ 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] @@ -625,16 +626,16 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/supplier/1.parquet, /testdata/tpch/data/supplier/10.parquet, /testdata/tpch/data/supplier/11.parquet], [/testdata/tpch/data/supplier/12.parquet, /testdata/tpch/data/supplier/13.parquet, /testdata/tpch/data/supplier/14.parquet], [/testdata/tpch/data/supplier/15.parquet, /testdata/tpch/data/supplier/16.parquet, /testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet, /testdata/tpch/data/supplier/4.parquet, /testdata/tpch/data/supplier/5.parquet], [/testdata/tpch/data/supplier/6.parquet, /testdata/tpch/data/supplier/7.parquet, /testdata/tpch/data/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/partsupp/1.parquet:.., /testdata/tpch/data/partsupp/10.parquet:.., /testdata/tpch/data/partsupp/11.parquet:..], [/testdata/tpch/data/partsupp/11.parquet:.., /testdata/tpch/data/partsupp/12.parquet:.., /testdata/tpch/data/partsupp/13.parquet:.., /testdata/tpch/data/partsupp/14.parquet:..], [/testdata/tpch/data/partsupp/14.parquet:.., /testdata/tpch/data/partsupp/15.parquet:.., /testdata/tpch/data/partsupp/16.parquet:.., /testdata/tpch/data/partsupp/2.parquet:..], [/testdata/tpch/data/partsupp/2.parquet:.., /testdata/tpch/data/partsupp/3.parquet:.., /testdata/tpch/data/partsupp/4.parquet:..], [/testdata/tpch/data/partsupp/4.parquet:.., /testdata/tpch/data/partsupp/5.parquet:.., /testdata/tpch/data/partsupp/6.parquet:.., /testdata/tpch/data/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ ps_suppkey@1 >= 1 AND ps_suppkey@1 <= 1000 OR ps_suppkey@1 >= 1 AND ps_suppkey@1 <= 1000 OR ps_suppkey@1 >= 1 AND ps_suppkey@1 <= 1000 OR ps_suppkey@1 >= 1 AND ps_suppkey@1 <= 1000 OR ps_suppkey@1 >= 1 AND ps_suppkey@1 <= 1000 OR ps_suppkey@1 >= 1 AND ps_suppkey@1 <= 1000 ], pruning_predicate=ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_max@0 >= 1 AND ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_min@3 <= 1000 OR ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_max@0 >= 1 AND ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_min@3 <= 1000 OR ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_max@0 >= 1 AND ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_min@3 <= 1000 OR ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_max@0 >= 1 AND ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_min@3 <= 1000 OR ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_max@0 >= 1 AND ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_min@3 <= 1000 OR ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_max@0 >= 1 AND ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_min@3 <= 1000, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.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,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 2 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.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)] └────────────────────────────────────────────────── "); @@ -645,37 +646,37 @@ mod tests { async fn test_tpch_12() -> Result<(), Box> { let plan = test_tpch_query(12).await?; assert_snapshot!(plan, @r#" - ┌───── Stage 5 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST] - │ NetworkCoalesceExec read_from=Stage 4, output_partitions=24, input_tasks=4 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] │ SortExec: expr=[l_shipmode@0 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[l_shipmode@0 as l_shipmode, sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@1 as high_line_count, sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@2 as low_line_count] │ AggregateExec: mode=FinalPartitioned, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 3 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] │ RepartitionExec: partitioning=Hash([l_shipmode@0], 24), input_partitions=6 │ 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)] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 1, output_partitions=6, n_tasks=3, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=3 │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 2, output_partitions=6, n_tasks=3, input_tasks=3 + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] │ RepartitionExec: partitioning=Hash([l_orderkey@0], 18), input_partitions=2 │ CoalesceBatchesExec: target_batch_size=8192 │ 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 Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], file_type=parquet, predicate=(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, 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 2 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] │ RepartitionExec: partitioning=Hash([o_orderkey@0], 18), input_partitions=2 - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderpriority], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── "#); @@ -686,39 +687,39 @@ mod tests { async fn test_tpch_13() -> Result<(), Box> { let plan = test_tpch_query(13).await?; assert_snapshot!(plan, @r" - ┌───── Stage 5 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [custdist@1 DESC, c_count@0 DESC] - │ NetworkCoalesceExec read_from=Stage 4, output_partitions=24, input_tasks=4 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] │ SortExec: expr=[custdist@1 DESC, c_count@0 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[c_count@0 as c_count, count(Int64(1))@1 as custdist] │ AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(Int64(1))] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 3 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] │ RepartitionExec: partitioning=Hash([c_count@0], 24), input_partitions=6 │ AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(Int64(1))] │ ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count] │ AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 2, output_partitions=6, n_tasks=3, input_tasks=1 + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 2 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] │ RepartitionExec: partitioning=Hash([c_custkey@0], 18), input_partitions=6 │ AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] │ ProjectionExec: expr=[c_custkey@1 as c_custkey, o_orderkey@0 as o_orderkey] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Right, on=[(o_custkey@1, c_custkey@0)], projection=[o_orderkey@0, c_custkey@2] │ CoalescePartitionsExec - │ NetworkCoalesceExec read_from=Stage 1, output_partitions=8, input_tasks=4 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_comment], file_type=parquet, predicate=o_comment@2 NOT LIKE %special%requests% └────────────────────────────────────────────────── "); @@ -729,32 +730,32 @@ mod tests { async fn test_tpch_14() -> Result<(), Box> { let plan = test_tpch_query(14).await?; assert_snapshot!(plan, @r#" - ┌───── Stage 4 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[100 * CAST(sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END)@0 AS Float64) / CAST(sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 AS Float64) as promo_revenue] │ AggregateExec: mode=Final, gby=[], aggr=[sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ CoalescePartitionsExec - │ NetworkCoalesceExec read_from=Stage 3, output_partitions=24, input_tasks=4 + │ [Stage 3] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 3 Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] │ 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] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], projection=[p_type@1, l_extendedprice@3, l_discount@4] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 1, output_partitions=6, n_tasks=4, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=3 │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 2, output_partitions=6, n_tasks=4, input_tasks=3 + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] │ RepartitionExec: partitioning=Hash([p_partkey@0], 24), input_partitions=2 - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_type], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 2 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] │ RepartitionExec: partitioning=Hash([l_partkey@0], 24), input_partitions=2 │ CoalesceBatchesExec: target_batch_size=8192 │ 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 Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01 AND DynamicFilterPhysicalExpr [ true ], 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=[] └────────────────────────────────────────────────── "#); @@ -765,14 +766,14 @@ mod tests { async fn test_tpch_15() -> Result<(), Box> { let plan = test_tpch_query(15).await?; assert_snapshot!(plan, @r" - ┌───── Stage 4 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST] │ SortExec: expr=[s_suppkey@0 ASC NULLS LAST], preserve_partitioning=[true] │ CoalesceBatchesExec: target_batch_size=8192 │ 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] │ AggregateExec: mode=Final, gby=[], aggr=[max(revenue0.total_revenue)] │ CoalescePartitionsExec - │ NetworkCoalesceExec read_from=Stage 2, output_partitions=24, input_tasks=4 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 │ CoalesceBatchesExec: target_batch_size=8192 │ 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 @@ -780,29 +781,29 @@ mod tests { │ ProjectionExec: expr=[l_suppkey@0 as supplier_no, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] │ AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 3, output_partitions=6, n_tasks=1, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 2 Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] │ AggregateExec: mode=Partial, gby=[], aggr=[max(revenue0.total_revenue)] │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] │ AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 1, output_partitions=6, n_tasks=4, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] │ RepartitionExec: partitioning=Hash([l_suppkey@0], 24), input_partitions=2 │ AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 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 3 Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] │ RepartitionExec: partitioning=Hash([l_suppkey@0], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 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=[] └────────────────────────────────────────────────── "); @@ -813,48 +814,48 @@ mod tests { async fn test_tpch_16() -> Result<(), Box> { let plan = test_tpch_query(16).await?; assert_snapshot!(plan, @r#" - ┌───── Stage 6 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST] - │ NetworkCoalesceExec read_from=Stage 5, output_partitions=24, input_tasks=4 + │ [Stage 5] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 5 Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 5 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] │ SortExec: expr=[supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, count(alias1)@3 as supplier_cnt] │ AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[count(alias1)] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 4, output_partitions=6, n_tasks=4, input_tasks=3 + │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 4 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] │ RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2], 24), input_partitions=6 │ AggregateExec: mode=Partial, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[count(alias1)] │ AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, alias1@3 as alias1], aggr=[] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 3, output_partitions=6, n_tasks=3, input_tasks=1 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 3 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] │ RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 18), input_partitions=6 │ 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=[] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(s_suppkey@0, ps_suppkey@0)] │ CoalescePartitionsExec - │ NetworkCoalesceExec read_from=Stage 1, output_partitions=8, input_tasks=4 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, 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] │ CoalesceBatchesExec: target_batch_size=8192 │ 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 - │ NetworkCoalesceExec read_from=Stage 2, output_partitions=8, input_tasks=4 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/partsupp/1.parquet:.., /testdata/tpch/data/partsupp/10.parquet:.., /testdata/tpch/data/partsupp/11.parquet:..], [/testdata/tpch/data/partsupp/11.parquet:.., /testdata/tpch/data/partsupp/12.parquet:.., /testdata/tpch/data/partsupp/13.parquet:.., /testdata/tpch/data/partsupp/14.parquet:..], [/testdata/tpch/data/partsupp/14.parquet:.., /testdata/tpch/data/partsupp/15.parquet:.., /testdata/tpch/data/partsupp/16.parquet:.., /testdata/tpch/data/partsupp/2.parquet:..], [/testdata/tpch/data/partsupp/2.parquet:.., /testdata/tpch/data/partsupp/3.parquet:.., /testdata/tpch/data/partsupp/4.parquet:..], [/testdata/tpch/data/partsupp/4.parquet:.., /testdata/tpch/data/partsupp/5.parquet:.., /testdata/tpch/data/partsupp/6.parquet:.., /testdata/tpch/data/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/supplier/1.parquet, /testdata/tpch/data/supplier/10.parquet, /testdata/tpch/data/supplier/11.parquet], [/testdata/tpch/data/supplier/12.parquet, /testdata/tpch/data/supplier/13.parquet, /testdata/tpch/data/supplier/14.parquet], [/testdata/tpch/data/supplier/15.parquet, /testdata/tpch/data/supplier/16.parquet, /testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet, /testdata/tpch/data/supplier/4.parquet, /testdata/tpch/data/supplier/5.parquet], [/testdata/tpch/data/supplier/6.parquet, /testdata/tpch/data/supplier/7.parquet, /testdata/tpch/data/supplier/8.parquet], ...]}, projection=[s_suppkey, s_comment], file_type=parquet, predicate=s_comment@1 LIKE %Customer%Complaints% └────────────────────────────────────────────────── - ┌───── Stage 2 Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 2 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(49), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(14), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(23), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(45), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(19), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(3), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(36), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(9), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_brand, p_type, p_size], file_type=parquet, predicate=p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(49), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(14), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(23), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(45), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(19), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(3), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(36), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(9), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), 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)] └────────────────────────────────────────────────── "#); @@ -865,42 +866,42 @@ mod tests { async fn test_tpch_17() -> Result<(), Box> { let plan = test_tpch_query(17).await?; assert_snapshot!(plan, @r" - ┌───── Stage 5 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[CAST(sum(lineitem.l_extendedprice)@0 AS Float64) / 7 as avg_yearly] │ AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice)] │ CoalescePartitionsExec - │ NetworkCoalesceExec read_from=Stage 4, output_partitions=24, input_tasks=4 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice)] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, 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] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 2, output_partitions=6, n_tasks=4, input_tasks=1 + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1 │ 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)] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 2 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] │ RepartitionExec: partitioning=Hash([p_partkey@2], 24), input_partitions=6 │ ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, p_partkey@0 as p_partkey] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], projection=[p_partkey@0, l_quantity@2, l_extendedprice@3] │ CoalescePartitionsExec - │ NetworkCoalesceExec read_from=Stage 1, output_partitions=8, input_tasks=4 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_quantity, l_extendedprice], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_brand, p_container], file_type=parquet, predicate=p_brand@1 = Brand#23 AND p_container@2 = 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,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] │ RepartitionExec: partitioning=Hash([l_partkey@0], 24), input_partitions=2 │ AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_quantity], file_type=parquet └────────────────────────────────────────────────── "); @@ -911,65 +912,65 @@ mod tests { async fn test_tpch_18() -> Result<(), Box> { let plan = test_tpch_query(18).await?; assert_snapshot!(plan, @r" - ┌───── Stage 9 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST] - │ NetworkCoalesceExec read_from=Stage 8, output_partitions=24, input_tasks=4 + │ [Stage 8] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 8 Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 8 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] │ SortExec: expr=[o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST], preserve_partitioning=[true] │ AggregateExec: mode=FinalPartitioned, gby=[c_name@0 as c_name, c_custkey@1 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@3 as o_orderdate, o_totalprice@4 as o_totalprice], aggr=[sum(lineitem.l_quantity)] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 7, output_partitions=6, n_tasks=4, input_tasks=1 + │ [Stage 7] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 7 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 7 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] │ RepartitionExec: partitioning=Hash([c_name@0, c_custkey@1, o_orderkey@2, o_orderdate@3, o_totalprice@4], 24), input_partitions=6 │ 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)] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(l_orderkey@0, o_orderkey@2)] │ CoalescePartitionsExec - │ NetworkCoalesceExec read_from=Stage 2, output_partitions=24, input_tasks=4 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, 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] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 5, output_partitions=6, n_tasks=1, input_tasks=3 + │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=3 │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 6, output_partitions=6, n_tasks=1, input_tasks=3 + │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 2 Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] │ CoalesceBatchesExec: target_batch_size=8192 │ 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)] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 1, output_partitions=6, n_tasks=4, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] │ RepartitionExec: partitioning=Hash([l_orderkey@0], 24), input_partitions=2 │ AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_quantity], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 5 Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 5 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] │ RepartitionExec: partitioning=Hash([o_orderkey@2], 6), input_partitions=6 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@4, o_orderdate@5] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 3, output_partitions=6, n_tasks=3, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3 │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 4, output_partitions=6, n_tasks=3, input_tasks=3 + │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 3 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] │ RepartitionExec: partitioning=Hash([c_custkey@0], 18), input_partitions=2 - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 4 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] │ RepartitionExec: partitioning=Hash([o_custkey@1], 18), input_partitions=2 - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── - ┌───── Stage 6 Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 6 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] │ RepartitionExec: partitioning=Hash([l_orderkey@0], 6), input_partitions=2 - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_quantity], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── "); @@ -980,7 +981,7 @@ mod tests { async fn test_tpch_19() -> Result<(), Box> { let plan = test_tpch_query(19).await?; assert_snapshot!(plan, @r#" - ┌───── Stage 2 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@0 as revenue] │ AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ CoalescePartitionsExec @@ -988,15 +989,15 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], filter=p_brand@1 = Brand#12 AND p_container@3 IN ([Literal { value: Utf8View("SM CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) 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 ([Literal { value: Utf8View("MED BAG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) 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 ([Literal { value: Utf8View("LG CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) 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 - │ NetworkCoalesceExec read_from=Stage 1, output_partitions=8, input_tasks=4 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ CoalesceBatchesExec: target_batch_size=8192 │ 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] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], file_type=parquet, predicate=(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 AND DynamicFilterPhysicalExpr [ l_partkey@0 >= 55 AND l_partkey@0 <= 19916 OR l_partkey@0 >= 55 AND l_partkey@0 <= 19916 OR l_partkey@0 >= 55 AND l_partkey@0 <= 19916 OR l_partkey@0 >= 55 AND l_partkey@0 <= 19916 OR l_partkey@0 >= 55 AND l_partkey@0 <= 19916 OR l_partkey@0 >= 55 AND l_partkey@0 <= 19916 ], 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 AND (l_partkey_null_count@11 != row_count@2 AND l_partkey_max@10 >= 55 AND l_partkey_null_count@11 != row_count@2 AND l_partkey_min@12 <= 19916 OR l_partkey_null_count@11 != row_count@2 AND l_partkey_max@10 >= 55 AND l_partkey_null_count@11 != row_count@2 AND l_partkey_min@12 <= 19916 OR l_partkey_null_count@11 != row_count@2 AND l_partkey_max@10 >= 55 AND l_partkey_null_count@11 != row_count@2 AND l_partkey_min@12 <= 19916 OR l_partkey_null_count@11 != row_count@2 AND l_partkey_max@10 >= 55 AND l_partkey_null_count@11 != row_count@2 AND l_partkey_min@12 <= 19916 OR l_partkey_null_count@11 != row_count@2 AND l_partkey_max@10 >= 55 AND l_partkey_null_count@11 != row_count@2 AND l_partkey_min@12 <= 19916 OR l_partkey_null_count@11 != row_count@2 AND l_partkey_max@10 >= 55 AND l_partkey_null_count@11 != row_count@2 AND l_partkey_min@12 <= 19916), required_guarantees=[l_shipinstruct in (DELIVER IN PERSON), l_shipmode in (AIR, AIR REG)] └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: (p_brand@1 = Brand#12 AND p_container@3 IN ([Literal { value: Utf8View("SM CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN ([Literal { value: Utf8View("MED BAG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN ([Literal { value: Utf8View("LG CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 15) AND p_size@2 >= 1 - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_brand, p_size, p_container], file_type=parquet, predicate=(p_brand@1 = Brand#12 AND p_container@3 IN ([Literal { value: Utf8View("SM CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN ([Literal { value: Utf8View("MED BAG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN ([Literal { value: Utf8View("LG CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 15) AND p_size@2 >= 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)] └────────────────────────────────────────────────── "#); @@ -1007,7 +1008,7 @@ mod tests { async fn test_tpch_20() -> Result<(), Box> { let plan = test_tpch_query(20).await?; assert_snapshot!(plan, @r" - ┌───── Stage 4 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] │ SortExec: expr=[s_name@0 ASC NULLS LAST], preserve_partitioning=[true] │ CoalesceBatchesExec: target_batch_size=8192 @@ -1016,7 +1017,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[s_suppkey@1, s_name@2, s_address@3] │ CoalescePartitionsExec - │ NetworkCoalesceExec read_from=Stage 1, output_partitions=8, input_tasks=4 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/supplier/1.parquet, /testdata/tpch/data/supplier/10.parquet, /testdata/tpch/data/supplier/11.parquet], [/testdata/tpch/data/supplier/12.parquet, /testdata/tpch/data/supplier/13.parquet, /testdata/tpch/data/supplier/14.parquet], [/testdata/tpch/data/supplier/15.parquet, /testdata/tpch/data/supplier/16.parquet, /testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet, /testdata/tpch/data/supplier/4.parquet, /testdata/tpch/data/supplier/5.parquet], [/testdata/tpch/data/supplier/6.parquet, /testdata/tpch/data/supplier/7.parquet, /testdata/tpch/data/supplier/8.parquet], ...]}, projection=[s_suppkey, s_name, s_address, s_nationkey], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ s_nationkey@3 >= 3 AND s_nationkey@3 <= 3 OR s_nationkey@3 >= 3 AND s_nationkey@3 <= 3 OR s_nationkey@3 >= 3 AND s_nationkey@3 <= 3 OR s_nationkey@3 >= 3 AND s_nationkey@3 <= 3 OR s_nationkey@3 >= 3 AND s_nationkey@3 <= 3 OR s_nationkey@3 >= 3 AND s_nationkey@3 <= 3 ], pruning_predicate=s_nationkey_null_count@1 != row_count@2 AND s_nationkey_max@0 >= 3 AND s_nationkey_null_count@1 != row_count@2 AND s_nationkey_min@3 <= 3 OR s_nationkey_null_count@1 != row_count@2 AND s_nationkey_max@0 >= 3 AND s_nationkey_null_count@1 != row_count@2 AND s_nationkey_min@3 <= 3 OR s_nationkey_null_count@1 != row_count@2 AND s_nationkey_max@0 >= 3 AND s_nationkey_null_count@1 != row_count@2 AND s_nationkey_min@3 <= 3 OR s_nationkey_null_count@1 != row_count@2 AND s_nationkey_max@0 >= 3 AND s_nationkey_null_count@1 != row_count@2 AND s_nationkey_min@3 <= 3 OR s_nationkey_null_count@1 != row_count@2 AND s_nationkey_max@0 >= 3 AND s_nationkey_null_count@1 != row_count@2 AND s_nationkey_min@3 <= 3 OR s_nationkey_null_count@1 != row_count@2 AND s_nationkey_max@0 >= 3 AND s_nationkey_null_count@1 != row_count@2 AND s_nationkey_min@3 <= 3, required_guarantees=[] │ CoalesceBatchesExec: target_batch_size=8192 │ 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] @@ -1024,31 +1025,31 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(p_partkey@0, ps_partkey@0)] │ CoalescePartitionsExec - │ NetworkCoalesceExec read_from=Stage 2, output_partitions=8, input_tasks=4 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/partsupp/1.parquet:.., /testdata/tpch/data/partsupp/10.parquet:.., /testdata/tpch/data/partsupp/11.parquet:..], [/testdata/tpch/data/partsupp/11.parquet:.., /testdata/tpch/data/partsupp/12.parquet:.., /testdata/tpch/data/partsupp/13.parquet:.., /testdata/tpch/data/partsupp/14.parquet:..], [/testdata/tpch/data/partsupp/14.parquet:.., /testdata/tpch/data/partsupp/15.parquet:.., /testdata/tpch/data/partsupp/16.parquet:.., /testdata/tpch/data/partsupp/2.parquet:..], [/testdata/tpch/data/partsupp/2.parquet:.., /testdata/tpch/data/partsupp/3.parquet:.., /testdata/tpch/data/partsupp/4.parquet:..], [/testdata/tpch/data/partsupp/4.parquet:.., /testdata/tpch/data/partsupp/5.parquet:.., /testdata/tpch/data/partsupp/6.parquet:.., /testdata/tpch/data/partsupp/7.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)] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 3, output_partitions=6, n_tasks=1, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.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 2 Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 2 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.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,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] │ RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1994-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1995-01-01, required_guarantees=[] └────────────────────────────────────────────────── "); @@ -1059,18 +1060,18 @@ mod tests { async fn test_tpch_21() -> Result<(), Box> { let plan = test_tpch_query(21).await?; assert_snapshot!(plan, @r" - ┌───── Stage 5 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [numwait@1 DESC, s_name@0 ASC NULLS LAST] - │ NetworkCoalesceExec read_from=Stage 4, output_partitions=24, input_tasks=4 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] │ SortExec: expr=[numwait@1 DESC, s_name@0 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[s_name@0 as s_name, count(Int64(1))@1 as numwait] │ AggregateExec: mode=FinalPartitioned, gby=[s_name@0 as s_name], aggr=[count(Int64(1))] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=1 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 3 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] │ RepartitionExec: partitioning=Hash([s_name@0], 24), input_partitions=6 │ AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[count(Int64(1))] │ CoalesceBatchesExec: target_batch_size=8192 @@ -1082,11 +1083,11 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@1)], projection=[s_name@1, l_orderkey@3, l_suppkey@4] │ CoalescePartitionsExec - │ NetworkCoalesceExec read_from=Stage 1, output_partitions=8, input_tasks=4 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ CoalesceBatchesExec: target_batch_size=8192 │ 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 - │ NetworkCoalesceExec read_from=Stage 2, output_partitions=8, input_tasks=4 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ CoalesceBatchesExec: target_batch_size=8192 │ 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 @@ -1099,16 +1100,16 @@ mod tests { │ FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@3 > l_commitdate@2 └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.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,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 2 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderstatus], file_type=parquet, predicate=o_orderstatus@1 = 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)] └────────────────────────────────────────────────── "); @@ -1119,42 +1120,42 @@ mod tests { async fn test_tpch_22() -> Result<(), Box> { let plan = test_tpch_query(22).await?; assert_snapshot!(plan, @r#" - ┌───── Stage 5 Tasks: t0:[p0] + ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST] - │ NetworkCoalesceExec read_from=Stage 4, output_partitions=24, input_tasks=4 + │ [Stage 4] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] │ SortExec: expr=[cntrycode@0 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[cntrycode@0 as cntrycode, count(Int64(1))@1 as numcust, sum(custsale.c_acctbal)@2 as totacctbal] │ AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)] │ CoalesceBatchesExec: target_batch_size=8192 - │ NetworkShuffleExec read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=1 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 3 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] │ RepartitionExec: partitioning=Hash([cntrycode@0], 24), input_partitions=6 │ AggregateExec: mode=Partial, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)] │ ProjectionExec: expr=[substr(c_phone@0, 1, 2) as cntrycode, c_acctbal@1 as c_acctbal] │ NestedLoopJoinExec: join_type=Inner, filter=CAST(c_acctbal@0 AS Decimal128(19, 6)) > avg(customer.c_acctbal)@1, projection=[c_phone@1, c_acctbal@2] │ AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)] │ CoalescePartitionsExec - │ NetworkCoalesceExec read_from=Stage 1, output_partitions=8, input_tasks=4 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)], projection=[c_phone@1, c_acctbal@2] │ CoalescePartitionsExec - │ NetworkCoalesceExec read_from=Stage 2, output_partitions=8, input_tasks=4 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_custkey], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 1 Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN ([Literal { value: Utf8View("13"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("31"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("23"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("29"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("30"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("18"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("17"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), projection=[c_acctbal@1] - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_phone, c_acctbal], file_type=parquet, predicate=c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN ([Literal { value: Utf8View("13"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("31"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("23"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("29"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("30"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("18"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("17"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), pruning_predicate=c_acctbal_null_count@1 != row_count@2 AND c_acctbal_max@0 > Some(0),15,2, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 2 Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 2 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: substr(c_phone@1, 1, 2) IN ([Literal { value: Utf8View("13"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("31"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("23"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("29"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("30"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("18"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("17"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) - │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_phone, c_acctbal], file_type=parquet, predicate=substr(c_phone@1, 1, 2) IN ([Literal { value: Utf8View("13"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("31"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("23"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("29"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("30"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("18"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("17"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) └────────────────────────────────────────────────── "#); @@ -1268,8 +1269,7 @@ mod tests { "Query {} results differ between executions", query_id ); - let plan_display = displayable(plan_d.as_ref()).indent(true).to_string(); - Ok(plan_display) + Ok(display_plan_ascii(plan_d.as_ref())) } pub fn get_test_data_dir() -> std::path::PathBuf {