diff --git a/src/common/mod.rs b/src/common/mod.rs index 3104cc9..85686e4 100644 --- a/src/common/mod.rs +++ b/src/common/mod.rs @@ -1,6 +1,5 @@ mod composed_extension_codec; #[allow(unused)] pub mod ttl_map; -pub mod util; pub(crate) use composed_extension_codec::ComposedPhysicalExtensionCodec; diff --git a/src/common/util.rs b/src/common/util.rs deleted file mode 100644 index 085c5c2..0000000 --- a/src/common/util.rs +++ /dev/null @@ -1,36 +0,0 @@ -use datafusion::error::Result; -use datafusion::physical_plan::{displayable, ExecutionPlan, ExecutionPlanProperties}; - -use std::fmt::Write; - -pub fn display_plan_with_partition_in_out(plan: &dyn ExecutionPlan) -> Result { - let mut f = String::new(); - - fn visit(plan: &dyn ExecutionPlan, indent: usize, f: &mut String) -> Result<()> { - let output_partitions = plan.output_partitioning().partition_count(); - let input_partitions = plan - .children() - .first() - .map(|child| child.output_partitioning().partition_count()); - - write!( - f, - "partitions [out:{:<3}{}]{} {}", - output_partitions, - input_partitions - .map(|p| format!("<-- in:{:<3}", p)) - .unwrap_or(" ".to_string()), - " ".repeat(indent), - displayable(plan).one_line() - )?; - - plan.children() - .iter() - .try_for_each(|input| visit(input.as_ref(), indent + 2, f))?; - - Ok(()) - } - - visit(plan, 0, &mut f)?; - Ok(f) -} diff --git a/src/physical_optimizer.rs b/src/physical_optimizer.rs index eddc646..7dfeeb3 100644 --- a/src/physical_optimizer.rs +++ b/src/physical_optimizer.rs @@ -227,8 +227,8 @@ mod tests { let query = r#"SELECT * FROM weather"#; let plan = sql_to_explain(query).await.unwrap(); assert_snapshot!(plan, @r" - ┌───── Stage 1 Task: partitions: 0,unassigned] - │partitions [out:1 ] DataSourceExec: file_groups={1 group: [[/testdata/weather.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 + ┌───── Stage 1 Tasks: t0:[p0] + │ DataSourceExec: file_groups={1 group: [[/testdata/weather.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 └────────────────────────────────────────────────── "); } @@ -239,23 +239,23 @@ mod tests { r#"SELECT count(*), "RainToday" FROM weather GROUP BY "RainToday" ORDER BY count(*)"#; let plan = sql_to_explain(query).await.unwrap(); assert_snapshot!(plan, @r" - ┌───── Stage 3 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:1 ] ProjectionExec: expr=[count(*)@0 as count(*), RainToday@1 as RainToday] - │partitions [out:1 <-- in:4 ] SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] - │partitions [out:4 <-- in:4 ] SortExec: expr=[count(*)@0 ASC NULLS LAST], preserve_partitioning=[true] - │partitions [out:4 <-- in:4 ] ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday, count(Int64(1))@1 as count(Int64(1))] - │partitions [out:4 <-- in:4 ] AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] - │partitions [out:4 <-- in:4 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:4 ] ArrowFlightReadExec: Stage 2 + ┌───── Stage 3 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 + │ ArrowFlightReadExec input_stage=2, input_partitions=4, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 2 Task: partitions: 0..3,unassigned] - │partitions [out:4 <-- in:4 ] RepartitionExec: partitioning=Hash([RainToday@0], 4), input_partitions=4 - │partitions [out:4 ] ArrowFlightReadExec: Stage 1 + ┌───── Stage 2 Tasks: t0:[p0,p1,p2,p3] + │ RepartitionExec: partitioning=Hash([RainToday@0], 4), input_partitions=4 + │ ArrowFlightReadExec input_stage=1, input_partitions=4, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 1 Task: partitions: 0..3,unassigned] - │partitions [out:4 <-- in:1 ] RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 - │partitions [out:1 <-- in:1 ] AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] - │partitions [out:1 ] DataSourceExec: file_groups={1 group: [[/testdata/weather.parquet]]}, projection=[RainToday], file_type=parquet + ┌───── Stage 1 Tasks: t0:[p0,p1,p2,p3] + │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 + │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] + │ DataSourceExec: file_groups={1 group: [[/testdata/weather.parquet]]}, projection=[RainToday], file_type=parquet └────────────────────────────────────────────────── "); } @@ -266,25 +266,25 @@ mod tests { r#"SELECT count(*), "RainToday" FROM weather GROUP BY "RainToday" ORDER BY count(*)"#; let plan = sql_to_explain_partitions_per_task(query, 2).await.unwrap(); assert_snapshot!(plan, @r" - ┌───── Stage 3 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:1 ] ProjectionExec: expr=[count(*)@0 as count(*), RainToday@1 as RainToday] - │partitions [out:1 <-- in:4 ] SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] - │partitions [out:4 <-- in:4 ] SortExec: expr=[count(*)@0 ASC NULLS LAST], preserve_partitioning=[true] - │partitions [out:4 <-- in:4 ] ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday, count(Int64(1))@1 as count(Int64(1))] - │partitions [out:4 <-- in:4 ] AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] - │partitions [out:4 <-- in:4 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:4 ] ArrowFlightReadExec: Stage 2 + ┌───── Stage 3 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 + │ ArrowFlightReadExec input_stage=2, input_partitions=4, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 Task: partitions: 0,1,unassigned],Task: partitions: 2,3,unassigned] - │partitions [out:4 <-- in:2 ] RepartitionExec: partitioning=Hash([RainToday@0], 4), input_partitions=2 - │partitions [out:2 <-- in:4 ] PartitionIsolatorExec [providing upto 2 partitions] - │partitions [out:4 ] ArrowFlightReadExec: Stage 1 + ┌───── Stage 2 Tasks: t0:[p0,p1] t1:[p2,p3] + │ RepartitionExec: partitioning=Hash([RainToday@0], 4), input_partitions=2 + │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ ArrowFlightReadExec input_stage=1, input_partitions=4, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 Task: partitions: 0,1,unassigned],Task: partitions: 2,3,unassigned] - │partitions [out:4 <-- in:2 ] RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 - │partitions [out:2 <-- in:1 ] PartitionIsolatorExec [providing upto 2 partitions] - │partitions [out:1 <-- in:1 ] AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] - │partitions [out:1 ] DataSourceExec: file_groups={1 group: [[/testdata/weather.parquet]]}, projection=[RainToday], file_type=parquet + ┌───── Stage 1 Tasks: t0:[p0,p1] t1:[p2,p3] + │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 + │ PartitionIsolatorExec Tasks: t0:[p0,p1,__,__] t1:[__,__,p0,p1] + │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] + │ DataSourceExec: file_groups={1 group: [[/testdata/weather.parquet]]}, projection=[RainToday], file_type=parquet └────────────────────────────────────────────────── "); } @@ -294,11 +294,11 @@ 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).await.unwrap(); assert_snapshot!(plan, @r" - ┌───── Stage 1 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:1 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:1 <-- in:1 ] HashJoinExec: mode=Partitioned, join_type=Left, on=[(RainToday@1, RainToday@1)], projection=[MinTemp@0, MaxTemp@2] - │partitions [out:1 ] DataSourceExec: file_groups={1 group: [[/testdata/weather.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet - │partitions [out:1 ] DataSourceExec: file_groups={1 group: [[/testdata/weather.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet + ┌───── Stage 1 Tasks: t0:[p0] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=Partitioned, join_type=Left, on=[(RainToday@1, RainToday@1)], projection=[MinTemp@0, MaxTemp@2] + │ DataSourceExec: file_groups={1 group: [[/testdata/weather.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet + │ DataSourceExec: file_groups={1 group: [[/testdata/weather.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet └────────────────────────────────────────────────── "); } @@ -331,40 +331,40 @@ mod tests { "#; let plan = sql_to_explain(query).await.unwrap(); assert_snapshot!(plan, @r" - ┌───── Stage 5 Task: partitions: 0..3,unassigned] - │partitions [out:4 <-- in:4 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:4 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Left, on=[(RainTomorrow@1, RainTomorrow@1)], projection=[MinTemp@0, MaxTemp@2] - │partitions [out:1 <-- in:4 ] CoalescePartitionsExec - │partitions [out:4 <-- in:4 ] ProjectionExec: expr=[avg(weather.MinTemp)@1 as MinTemp, RainTomorrow@0 as RainTomorrow] - │partitions [out:4 <-- in:4 ] AggregateExec: mode=FinalPartitioned, gby=[RainTomorrow@0 as RainTomorrow], aggr=[avg(weather.MinTemp)] - │partitions [out:4 <-- in:4 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:4 ] ArrowFlightReadExec: Stage 2 - │partitions [out:4 <-- in:4 ] ProjectionExec: expr=[avg(weather.MaxTemp)@1 as MaxTemp, RainTomorrow@0 as RainTomorrow] - │partitions [out:4 <-- in:4 ] AggregateExec: mode=FinalPartitioned, gby=[RainTomorrow@0 as RainTomorrow], aggr=[avg(weather.MaxTemp)] - │partitions [out:4 <-- in:4 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:4 ] ArrowFlightReadExec: Stage 4 + ┌───── Stage 5 Tasks: t0:[p0,p1,p2,p3] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(RainTomorrow@1, RainTomorrow@1)], projection=[MinTemp@0, MaxTemp@2] + │ CoalescePartitionsExec + │ ProjectionExec: expr=[avg(weather.MinTemp)@1 as MinTemp, RainTomorrow@0 as RainTomorrow] + │ AggregateExec: mode=FinalPartitioned, gby=[RainTomorrow@0 as RainTomorrow], aggr=[avg(weather.MinTemp)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ ArrowFlightReadExec input_stage=2, input_partitions=4, input_tasks=1 + │ 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 + │ ArrowFlightReadExec input_stage=4, input_partitions=4, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 2 Task: partitions: 0..3,unassigned] - │partitions [out:4 <-- in:4 ] RepartitionExec: partitioning=Hash([RainTomorrow@0], 4), input_partitions=4 - │partitions [out:4 <-- in:4 ] AggregateExec: mode=Partial, gby=[RainTomorrow@1 as RainTomorrow], aggr=[avg(weather.MinTemp)] - │partitions [out:4 <-- in:4 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:4 <-- in:4 ] FilterExec: RainToday@1 = yes, projection=[MinTemp@0, RainTomorrow@2] - │partitions [out:4 ] ArrowFlightReadExec: Stage 1 + ┌───── Stage 2 Tasks: t0:[p0,p1,p2,p3] + │ RepartitionExec: partitioning=Hash([RainTomorrow@0], 4), 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] + │ ArrowFlightReadExec input_stage=1, input_partitions=4, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 1 Task: partitions: 0..3,unassigned] - │partitions [out:4 <-- in:1 ] RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 - │partitions [out:1 ] DataSourceExec: file_groups={1 group: [[/testdata/weather.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 1 Tasks: t0:[p0,p1,p2,p3] + │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 + │ DataSourceExec: file_groups={1 group: [[/testdata/weather.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 4 Task: partitions: 0..3,unassigned] - │partitions [out:4 <-- in:4 ] RepartitionExec: partitioning=Hash([RainTomorrow@0], 4), input_partitions=4 - │partitions [out:4 <-- in:4 ] AggregateExec: mode=Partial, gby=[RainTomorrow@1 as RainTomorrow], aggr=[avg(weather.MaxTemp)] - │partitions [out:4 <-- in:4 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:4 <-- in:4 ] FilterExec: RainToday@1 = no, projection=[MaxTemp@0, RainTomorrow@2] - │partitions [out:4 ] ArrowFlightReadExec: Stage 3 + ┌───── Stage 4 Tasks: t0:[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] + │ ArrowFlightReadExec input_stage=3, input_partitions=4, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 3 Task: partitions: 0..3,unassigned] - │partitions [out:4 <-- in:1 ] RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 - │partitions [out:1 ] DataSourceExec: file_groups={1 group: [[/testdata/weather.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)] + ┌───── Stage 3 Tasks: t0:[p0,p1,p2,p3] + │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 + │ DataSourceExec: file_groups={1 group: [[/testdata/weather.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)] └────────────────────────────────────────────────── "); } @@ -374,9 +374,9 @@ mod tests { let query = r#"SELECT * FROM weather ORDER BY "MinTemp" DESC "#; let plan = sql_to_explain(query).await.unwrap(); assert_snapshot!(plan, @r" - ┌───── Stage 1 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:1 ] SortExec: expr=[MinTemp@0 DESC], preserve_partitioning=[false] - │partitions [out:1 ] DataSourceExec: file_groups={1 group: [[/testdata/weather.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 + ┌───── Stage 1 Tasks: t0:[p0] + │ SortExec: expr=[MinTemp@0 DESC], preserve_partitioning=[false] + │ DataSourceExec: file_groups={1 group: [[/testdata/weather.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 └────────────────────────────────────────────────── "); } @@ -386,19 +386,19 @@ mod tests { let query = r#"SELECT DISTINCT "RainToday", "WindGustDir" FROM weather"#; let plan = sql_to_explain(query).await.unwrap(); assert_snapshot!(plan, @r" - ┌───── Stage 3 Task: partitions: 0..3,unassigned] - │partitions [out:4 <-- in:4 ] AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[] - │partitions [out:4 <-- in:4 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:4 ] ArrowFlightReadExec: Stage 2 + ┌───── Stage 3 Tasks: t0:[p0,p1,p2,p3] + │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[] + │ CoalesceBatchesExec: target_batch_size=8192 + │ ArrowFlightReadExec input_stage=2, input_partitions=4, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 2 Task: partitions: 0..3,unassigned] - │partitions [out:4 <-- in:4 ] RepartitionExec: partitioning=Hash([RainToday@0, WindGustDir@1], 4), input_partitions=4 - │partitions [out:4 ] ArrowFlightReadExec: Stage 1 + ┌───── Stage 2 Tasks: t0:[p0,p1,p2,p3] + │ RepartitionExec: partitioning=Hash([RainToday@0, WindGustDir@1], 4), input_partitions=4 + │ ArrowFlightReadExec input_stage=1, input_partitions=4, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 1 Task: partitions: 0..3,unassigned] - │partitions [out:4 <-- in:1 ] RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 - │partitions [out:1 <-- in:1 ] AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[] - │partitions [out:1 ] DataSourceExec: file_groups={1 group: [[/testdata/weather.parquet]]}, projection=[RainToday, WindGustDir], file_type=parquet + ┌───── Stage 1 Tasks: t0:[p0,p1,p2,p3] + │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 + │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[] + │ DataSourceExec: file_groups={1 group: [[/testdata/weather.parquet]]}, projection=[RainToday, WindGustDir], file_type=parquet └────────────────────────────────────────────────── "); } diff --git a/src/plan/arrow_flight_read.rs b/src/plan/arrow_flight_read.rs index d3b9ac5..700b959 100644 --- a/src/plan/arrow_flight_read.rs +++ b/src/plan/arrow_flight_read.rs @@ -102,12 +102,7 @@ impl ArrowFlightReadExec { impl DisplayAs for ArrowFlightReadExec { fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { - match self { - ArrowFlightReadExec::Pending(_) => write!(f, "ArrowFlightReadExec"), - ArrowFlightReadExec::Ready(v) => { - write!(f, "ArrowFlightReadExec: Stage {:<3}", v.stage_num) - } - } + write!(f, "ArrowFlightReadExec") } } diff --git a/src/plan/isolator.rs b/src/plan/isolator.rs index 0ebf82a..b7ea79f 100644 --- a/src/plan/isolator.rs +++ b/src/plan/isolator.rs @@ -46,11 +46,7 @@ impl PartitionIsolatorExec { impl DisplayAs for PartitionIsolatorExec { fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { - write!( - f, - "PartitionIsolatorExec [providing upto {} partitions]", - self.partition_count - ) + write!(f, "PartitionIsolatorExec",) } } diff --git a/src/stage/display.rs b/src/stage/display.rs index 794d45d..fdf0572 100644 --- a/src/stage/display.rs +++ b/src/stage/display.rs @@ -1,3 +1,15 @@ +use super::ExecutionStage; +use crate::plan::PartitionIsolatorExec; +use crate::{ + task::{format_pg, ExecutionTask}, + ArrowFlightReadExec, +}; +use datafusion::physical_plan::{displayable, ExecutionPlan, ExecutionPlanProperties}; +use datafusion::{ + error::Result, + physical_plan::{DisplayAs, DisplayFormatType}, +}; +use itertools::Itertools; /// Be able to display a nice tree for stages. /// /// The challenge to doing this at the moment is that `TreeRenderVistor` @@ -12,24 +24,47 @@ /// the Stage tree. use std::fmt::Write; -use datafusion::{ - error::Result, - physical_plan::{DisplayAs, DisplayFormatType}, -}; - -use crate::{ - common::util::display_plan_with_partition_in_out, - task::{format_pg, ExecutionTask}, -}; - -use super::ExecutionStage; - // Unicode box-drawing characters for creating borders and connections. const LTCORNER: &str = "┌"; // Left top corner const LDCORNER: &str = "└"; // Left bottom corner const VERTICAL: &str = "│"; // Vertical line const HORIZONTAL: &str = "─"; // Horizontal line +impl ExecutionStage { + 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(ArrowFlightReadExec::Ready(ready)) = + plan.as_any().downcast_ref::() + { + let Some(input_stage) = &self.child_stages_iter().find(|v| v.num == ready.stage_num) + else { + writeln!(f, "Wrong partition number {}", ready.stage_num)?; + return Ok(()); + }; + let tasks = input_stage.tasks.len(); + let partitions = plan.output_partitioning().partition_count(); + let stage = ready.stage_num; + write!( + f, + " input_stage={stage}, input_partitions={partitions}, input_tasks={tasks}", + )?; + } + + if plan.as_any().is::() { + write!(f, " {}", format_tasks_for_partition_isolator(&self.tasks))?; + } + writeln!(f)?; + + for child in plan.children() { + self.format(child.as_ref(), indent + 2, f)?; + } + Ok(()) + } +} + impl DisplayAs for ExecutionStage { fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result { #[allow(clippy::format_in_format_args)] @@ -44,10 +79,11 @@ impl DisplayAs for ExecutionStage { LTCORNER, HORIZONTAL.repeat(5), format!(" {} ", self.name), - format_tasks(&self.tasks), + format_tasks_for_stage(&self.tasks), )?; - let plan_str = display_plan_with_partition_in_out(self.plan.as_ref()) - .map_err(|_| std::fmt::Error {})?; + + 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()) @@ -186,10 +222,28 @@ pub fn display_stage_graphviz(stage: &ExecutionStage) -> Result { Ok(f) } -fn format_tasks(tasks: &[ExecutionTask]) -> String { - tasks - .iter() - .map(|task| format!("{task}")) - .collect::>() - .join(",") +fn format_tasks_for_stage(tasks: &[ExecutionTask]) -> String { + let mut result = "Tasks: ".to_string(); + for (i, t) in tasks.iter().enumerate() { + result += &format!("t{i}:["); + result += &t.partition_group.iter().map(|v| format!("p{v}")).join(","); + result += "] " + } + result +} + +fn format_tasks_for_partition_isolator(tasks: &[ExecutionTask]) -> String { + let mut result = "Tasks: ".to_string(); + let mut partitions = vec![]; + for t in tasks.iter() { + partitions.extend(vec!["__".to_string(); t.partition_group.len()]) + } + for (i, t) in tasks.iter().enumerate() { + let mut partitions = partitions.clone(); + for (i, p) in t.partition_group.iter().enumerate() { + partitions[*p as usize] = format!("p{i}") + } + result += &format!("t{i}:[{}] ", partitions.join(",")); + } + result } diff --git a/tests/custom_extension_codec.rs b/tests/custom_extension_codec.rs index 4b9fb43..7ee96a9 100644 --- a/tests/custom_extension_codec.rs +++ b/tests/custom_extension_codec.rs @@ -61,20 +61,20 @@ mod tests { DistributedPhysicalOptimizerRule::default().distribute_plan(distributed_plan)?; assert_snapshot!(displayable(&distributed_plan).indent(true).to_string(), @r" - ┌───── Stage 3 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:1 ] SortExec: expr=[numbers@0 DESC NULLS LAST], preserve_partitioning=[false] - │partitions [out:1 <-- in:10 ] RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=10 - │partitions [out:10 ] ArrowFlightReadExec: Stage 2 + ┌───── Stage 3 Tasks: t0:[p0] + │ SortExec: expr=[numbers@0 DESC NULLS LAST], preserve_partitioning=[false] + │ RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=10 + │ ArrowFlightReadExec input_stage=2, input_partitions=10, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 2 Task: partitions: 0..9,unassigned] - │partitions [out:10 <-- in:1 ] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - │partitions [out:1 <-- in:1 ] SortExec: expr=[numbers@0 DESC NULLS LAST], preserve_partitioning=[false] - │partitions [out:1 ] ArrowFlightReadExec: Stage 1 + ┌───── Stage 2 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] + │ RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + │ SortExec: expr=[numbers@0 DESC NULLS LAST], preserve_partitioning=[false] + │ ArrowFlightReadExec input_stage=1, input_partitions=1, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 1 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:1 ] RepartitionExec: partitioning=Hash([numbers@0], 1), input_partitions=1 - │partitions [out:1 <-- in:1 ] FilterExec: numbers@0 > 1 - │partitions [out:1 ] Int64ListExec: length=6 + ┌───── Stage 1 Tasks: t0:[p0] + │ RepartitionExec: partitioning=Hash([numbers@0], 1), input_partitions=1 + │ FilterExec: numbers@0 > 1 + │ Int64ListExec: length=6 └────────────────────────────────────────────────── "); diff --git a/tests/distributed_aggregation.rs b/tests/distributed_aggregation.rs index 4794a5f..a967ea1 100644 --- a/tests/distributed_aggregation.rs +++ b/tests/distributed_aggregation.rs @@ -50,25 +50,25 @@ mod tests { assert_snapshot!(physical_distributed_str, @r" - ┌───── Stage 3 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:1 ] ProjectionExec: expr=[count(*)@0 as count(*), RainToday@1 as RainToday] - │partitions [out:1 <-- in:3 ] SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] - │partitions [out:3 <-- in:3 ] SortExec: expr=[count(*)@0 ASC NULLS LAST], preserve_partitioning=[true] - │partitions [out:3 <-- in:3 ] ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday, count(Int64(1))@1 as count(Int64(1))] - │partitions [out:3 <-- in:3 ] AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 ] ArrowFlightReadExec: Stage 2 + ┌───── Stage 3 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 + │ ArrowFlightReadExec input_stage=2, input_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 2 Task: partitions: 0,unassigned],Task: partitions: 1,unassigned],Task: partitions: 2,unassigned] - │partitions [out:3 <-- in:1 ] RepartitionExec: partitioning=Hash([RainToday@0], 3), input_partitions=1 - │partitions [out:1 <-- in:3 ] PartitionIsolatorExec [providing upto 1 partitions] - │partitions [out:3 ] ArrowFlightReadExec: Stage 1 + ┌───── Stage 2 Tasks: t0:[p0] t1:[p1] t2:[p2] + │ RepartitionExec: partitioning=Hash([RainToday@0], 3), input_partitions=1 + │ PartitionIsolatorExec Tasks: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] + │ ArrowFlightReadExec input_stage=1, input_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 Task: partitions: 0,unassigned],Task: partitions: 1,unassigned],Task: partitions: 2,unassigned] - │partitions [out:3 <-- in:1 ] RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 - │partitions [out:1 <-- in:1 ] PartitionIsolatorExec [providing upto 1 partitions] - │partitions [out:1 <-- in:1 ] AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] - │partitions [out:1 ] DataSourceExec: file_groups={1 group: [[/testdata/weather.parquet]]}, projection=[RainToday], file_type=parquet + ┌───── Stage 1 Tasks: t0:[p0] t1:[p1] t2:[p2] + │ RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 + │ PartitionIsolatorExec Tasks: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] + │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] + │ DataSourceExec: file_groups={1 group: [[/testdata/weather.parquet]]}, projection=[RainToday], file_type=parquet └────────────────────────────────────────────────── ", ); diff --git a/tests/highly_distributed_query.rs b/tests/highly_distributed_query.rs index 9c13365..ddcbe62 100644 --- a/tests/highly_distributed_query.rs +++ b/tests/highly_distributed_query.rs @@ -14,6 +14,7 @@ mod tests { use std::sync::Arc; #[tokio::test] + #[ignore] // This test is flaky async fn highly_distributed_query() -> Result<(), Box> { let (ctx, _guard) = start_localhost_context(9, DefaultSessionBuilder).await; register_parquet_tables(&ctx).await?; @@ -45,20 +46,20 @@ mod tests { assert_snapshot!(physical_distributed_str, @r" - ┌───── Stage 4 Task: partitions: 0..4,unassigned] - │partitions [out:5 ] ArrowFlightReadExec: Stage 3 + ┌───── Stage 4 Tasks: t0:[p0,p1,p2,p3,p4] + │ ArrowFlightReadExec input_stage=3, input_partitions=5, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 3 Task: partitions: 0..4,unassigned] - │partitions [out:5 <-- in:10 ] RepartitionExec: partitioning=RoundRobinBatch(5), input_partitions=10 - │partitions [out:10 ] ArrowFlightReadExec: Stage 2 + ┌───── Stage 3 Tasks: t0:[p0,p1,p2,p3,p4] + │ RepartitionExec: partitioning=RoundRobinBatch(5), input_partitions=10 + │ ArrowFlightReadExec input_stage=2, input_partitions=10, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 2 Task: partitions: 0..9,unassigned] - │partitions [out:10 <-- in:1 ] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 - │partitions [out:1 ] ArrowFlightReadExec: Stage 1 + ┌───── Stage 2 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] + │ RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + │ ArrowFlightReadExec input_stage=1, input_partitions=1, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 1 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:1 ] RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=1 - │partitions [out: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 + ┌───── Stage 1 Tasks: t0:[p0] + │ RepartitionExec: partitioning=RoundRobinBatch(1), 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/tpch_validation_test.rs b/tests/tpch_validation_test.rs index ef1fe56..14364f4 100644 --- a/tests/tpch_validation_test.rs +++ b/tests/tpch_validation_test.rs @@ -19,22 +19,22 @@ mod tests { async fn test_tpch_1() -> Result<(), Box> { let plan = test_tpch_query(1).await?; assert_snapshot!(plan, @r" - ┌───── Stage 2 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:3 ] SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST] - │partitions [out:3 <-- in:3 ] SortExec: expr=[l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST], preserve_partitioning=[true] - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 <-- in:3 ] 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))] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 ] ArrowFlightReadExec: Stage 1 + ┌───── Stage 2 Tasks: t0:[p0] + │ SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST] + │ SortExec: expr=[l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus, sum(lineitem.l_quantity)@2 as sum_qty, sum(lineitem.l_extendedprice)@3 as sum_base_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@4 as sum_disc_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax)@5 as sum_charge, avg(lineitem.l_quantity)@6 as avg_qty, avg(lineitem.l_extendedprice)@7 as avg_price, avg(lineitem.l_discount)@8 as avg_disc, count(Int64(1))@9 as count_order] + │ AggregateExec: mode=FinalPartitioned, gby=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(Int64(1))] + │ CoalesceBatchesExec: target_batch_size=8192 + │ ArrowFlightReadExec input_stage=1, input_partitions=3, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 Task: partitions: 0,1,unassigned],Task: partitions: 2,unassigned] - │partitions [out:3 <-- in:2 ] RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 3), input_partitions=2 - │partitions [out:2 <-- in:3 ] PartitionIsolatorExec [providing upto 2 partitions] - │partitions [out:3 <-- in:3 ] 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))] - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.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=[] + ┌───── Stage 1 Tasks: t0:[p0,p1] t1:[p2] + │ RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 3), input_partitions=2 + │ PartitionIsolatorExec Tasks: t0:[p0,p1,__] t1:[__,__,p0] + │ 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] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.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=[] └────────────────────────────────────────────────── "); Ok(()) @@ -44,61 +44,61 @@ mod tests { async fn test_tpch_2() -> Result<(), Box> { let plan = test_tpch_query(2).await?; assert_snapshot!(plan, @r" - ┌───── Stage 2 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:3 ] SortPreservingMergeExec: [s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST] - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, 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] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] 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] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/region/1.parquet], [/testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.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)] - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] 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] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/nation/1.parquet], [/testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet]]}, projection=[n_nationkey, n_name, n_regionkey], file_type=parquet - │partitions [out:3 <-- in:3 ] ProjectionExec: expr=[p_partkey@6 as p_partkey, p_mfgr@7 as p_mfgr, s_name@0 as s_name, s_address@1 as s_address, s_nationkey@2 as s_nationkey, s_phone@3 as s_phone, s_acctbal@4 as s_acctbal, s_comment@5 as s_comment, ps_supplycost@8 as ps_supplycost] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@2)], projection=[s_name@1, s_address@2, s_nationkey@3, s_phone@4, s_acctbal@5, s_comment@6, p_partkey@7, p_mfgr@8, ps_supplycost@10] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/supplier/1.parquet], [/testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], file_type=parquet - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] 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] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@1] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/part/1.parquet], [/testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.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)] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/partsupp/1.parquet], [/testdata/tpch/data/partsupp/2.parquet], [/testdata/tpch/data/partsupp/3.parquet]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet - │partitions [out:3 <-- in:3 ] ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] - │partitions [out:3 <-- in:3 ] AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 ] ArrowFlightReadExec: Stage 1 + ┌───── Stage 2 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] + │ 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=CollectLeft, 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] + │ CoalescePartitionsExec + │ 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 + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/region/1.parquet], [/testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.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)] + │ 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] + │ CoalescePartitionsExec + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/nation/1.parquet], [/testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet]]}, projection=[n_nationkey, n_name, n_regionkey], file_type=parquet + │ ProjectionExec: expr=[p_partkey@6 as p_partkey, p_mfgr@7 as p_mfgr, s_name@0 as s_name, s_address@1 as s_address, s_nationkey@2 as s_nationkey, s_phone@3 as s_phone, s_acctbal@4 as s_acctbal, s_comment@5 as s_comment, ps_supplycost@8 as ps_supplycost] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@2)], projection=[s_name@1, s_address@2, s_nationkey@3, s_phone@4, s_acctbal@5, s_comment@6, p_partkey@7, p_mfgr@8, ps_supplycost@10] + │ CoalescePartitionsExec + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/supplier/1.parquet], [/testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], file_type=parquet + │ 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 + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@1] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/part/1.parquet], [/testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.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)] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/partsupp/1.parquet], [/testdata/tpch/data/partsupp/2.parquet], [/testdata/tpch/data/partsupp/3.parquet]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet + │ 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 + │ ArrowFlightReadExec input_stage=1, input_partitions=3, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 1 Task: partitions: 0..2,unassigned] - │partitions [out:3 <-- in:3 ] RepartitionExec: partitioning=Hash([ps_partkey@0], 3), input_partitions=3 - │partitions [out:3 <-- in:3 ] AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@2)], projection=[ps_partkey@1, ps_supplycost@2] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/region/1.parquet], [/testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.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)] - │partitions [out:3 <-- in:3 ] ProjectionExec: expr=[ps_partkey@1 as ps_partkey, ps_supplycost@2 as ps_supplycost, n_regionkey@0 as n_regionkey] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_regionkey@1, ps_partkey@2, ps_supplycost@3] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/nation/1.parquet], [/testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet]]}, projection=[n_nationkey, n_regionkey], file_type=parquet - │partitions [out:3 <-- in:3 ] ProjectionExec: expr=[ps_partkey@1 as ps_partkey, ps_supplycost@2 as ps_supplycost, s_nationkey@0 as s_nationkey] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@1)], projection=[s_nationkey@1, ps_partkey@2, ps_supplycost@4] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/supplier/1.parquet], [/testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet]]}, projection=[s_suppkey, s_nationkey], file_type=parquet - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/partsupp/1.parquet], [/testdata/tpch/data/partsupp/2.parquet], [/testdata/tpch/data/partsupp/3.parquet]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet + ┌───── Stage 1 Tasks: t0:[p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ps_partkey@0], 3), input_partitions=3 + │ 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 + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/region/1.parquet], [/testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.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)] + │ 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] + │ CoalescePartitionsExec + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/nation/1.parquet], [/testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet]]}, projection=[n_nationkey, n_regionkey], file_type=parquet + │ ProjectionExec: expr=[ps_partkey@1 as ps_partkey, 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@1)], projection=[s_nationkey@1, ps_partkey@2, ps_supplycost@4] + │ CoalescePartitionsExec + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/supplier/1.parquet], [/testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet]]}, projection=[s_suppkey, s_nationkey], file_type=parquet + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/partsupp/1.parquet], [/testdata/tpch/data/partsupp/2.parquet], [/testdata/tpch/data/partsupp/3.parquet]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet └────────────────────────────────────────────────── "); Ok(()) @@ -108,32 +108,32 @@ mod tests { async fn test_tpch_3() -> Result<(), Box> { let plan = test_tpch_query(3).await?; assert_snapshot!(plan, @r" - ┌───── Stage 2 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:3 ] SortPreservingMergeExec: [revenue@1 DESC, o_orderdate@2 ASC NULLS LAST] - │partitions [out:3 <-- in:3 ] SortExec: expr=[revenue@1 DESC, o_orderdate@2 ASC NULLS LAST], preserve_partitioning=[true] - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 <-- in:3 ] 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)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 ] ArrowFlightReadExec: Stage 1 + ┌───── Stage 2 Tasks: t0:[p0] + │ SortPreservingMergeExec: [revenue@1 DESC, o_orderdate@2 ASC NULLS LAST] + │ SortExec: expr=[revenue@1 DESC, o_orderdate@2 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[l_orderkey@0 as l_orderkey, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@3 as revenue, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority] + │ AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ ArrowFlightReadExec input_stage=1, input_partitions=3, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 1 Task: partitions: 0..2,unassigned] - │partitions [out:3 <-- in:3 ] RepartitionExec: partitioning=Hash([l_orderkey@0, o_orderdate@1, o_shippriority@2], 3), input_partitions=3 - │partitions [out:3 <-- in:3 ] 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)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, o_shippriority@2, l_orderkey@3, l_extendedprice@4, l_discount@5] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/customer/1.parquet], [/testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.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)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: o_orderdate@2 < 1995-03-15 - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/orders/1.parquet], [/testdata/tpch/data/orders/2.parquet], [/testdata/tpch/data/orders/3.parquet]]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], file_type=parquet, predicate=o_orderdate@2 < 1995-03-15, pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@0 < 1995-03-15, required_guarantees=[] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.parquet]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 > 1995-03-15, 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,p2] + │ RepartitionExec: partitioning=Hash([l_orderkey@0, o_orderdate@1, o_shippriority@2], 3), input_partitions=3 + │ 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 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, o_shippriority@2, l_orderkey@3, l_extendedprice@4, l_discount@5] + │ CoalescePartitionsExec + │ 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 + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/customer/1.parquet], [/testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.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)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: o_orderdate@2 < 1995-03-15 + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/orders/1.parquet], [/testdata/tpch/data/orders/2.parquet], [/testdata/tpch/data/orders/3.parquet]]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], file_type=parquet, predicate=o_orderdate@2 < 1995-03-15, pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@0 < 1995-03-15, required_guarantees=[] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.parquet]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 > 1995-03-15, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 > 1995-03-15, required_guarantees=[] └────────────────────────────────────────────────── "); Ok(()) @@ -143,26 +143,26 @@ mod tests { async fn test_tpch_4() -> Result<(), Box> { let plan = test_tpch_query(4).await?; assert_snapshot!(plan, @r" - ┌───── Stage 2 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:3 ] SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST] - │partitions [out:3 <-- in:3 ] SortExec: expr=[o_orderpriority@0 ASC NULLS LAST], preserve_partitioning=[true] - │partitions [out:3 <-- in:3 ] ProjectionExec: expr=[o_orderpriority@0 as o_orderpriority, count(Int64(1))@1 as order_count] - │partitions [out:3 <-- in:3 ] AggregateExec: mode=FinalPartitioned, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 ] ArrowFlightReadExec: Stage 1 + ┌───── Stage 2 Tasks: t0:[p0] + │ SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST] + │ 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 + │ ArrowFlightReadExec input_stage=1, input_partitions=3, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 1 Task: partitions: 0..2,unassigned] - │partitions [out:3 <-- in:3 ] RepartitionExec: partitioning=Hash([o_orderpriority@0], 3), input_partitions=3 - │partitions [out:3 <-- in:3 ] AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(l_orderkey@0, o_orderkey@0)], projection=[o_orderpriority@1] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.parquet]]}, projection=[l_orderkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@2 > l_commitdate@1 - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/orders/1.parquet], [/testdata/tpch/data/orders/2.parquet], [/testdata/tpch/data/orders/3.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,p2] + │ RepartitionExec: partitioning=Hash([o_orderpriority@0], 3), input_partitions=3 + │ 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 + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.parquet]]}, projection=[l_orderkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@2 > l_commitdate@1 + │ 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={3 groups: [[/testdata/tpch/data/orders/1.parquet], [/testdata/tpch/data/orders/2.parquet], [/testdata/tpch/data/orders/3.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=[] └────────────────────────────────────────────────── "); Ok(()) @@ -172,44 +172,44 @@ mod tests { async fn test_tpch_5() -> Result<(), Box> { let plan = test_tpch_query(5).await?; assert_snapshot!(plan, @r" - ┌───── Stage 2 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:3 ] SortPreservingMergeExec: [revenue@1 DESC] - │partitions [out:3 <-- in:3 ] SortExec: expr=[revenue@1 DESC], preserve_partitioning=[true] - │partitions [out:3 <-- in:3 ] ProjectionExec: expr=[n_name@0 as n_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as revenue] - │partitions [out:3 <-- in:3 ] AggregateExec: mode=FinalPartitioned, gby=[n_name@0 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 ] ArrowFlightReadExec: Stage 1 + ┌───── Stage 2 Tasks: t0:[p0] + │ SortPreservingMergeExec: [revenue@1 DESC] + │ SortExec: expr=[revenue@1 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[n_name@0 as n_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as revenue] + │ AggregateExec: mode=FinalPartitioned, gby=[n_name@0 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ ArrowFlightReadExec input_stage=1, input_partitions=3, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 1 Task: partitions: 0..2,unassigned] - │partitions [out:3 <-- in:3 ] RepartitionExec: partitioning=Hash([n_name@0], 3), input_partitions=3 - │partitions [out:3 <-- in:3 ] AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@3)], projection=[l_extendedprice@1, l_discount@2, n_name@3] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/region/1.parquet], [/testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.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)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@4, n_regionkey@5] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, s_nationkey@0 as s_nationkey] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1), (s_nationkey@1, c_nationkey@0)], projection=[s_nationkey@1, l_extendedprice@4, l_discount@5] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/supplier/1.parquet], [/testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet]]}, projection=[s_suppkey, s_nationkey], file_type=parquet - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@1, l_orderkey@0)], projection=[c_nationkey@0, l_suppkey@3, l_extendedprice@4, l_discount@5] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] ProjectionExec: expr=[c_nationkey@1 as c_nationkey, o_orderkey@0 as o_orderkey] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_custkey@1, c_custkey@0)], projection=[o_orderkey@0, c_nationkey@3] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/orders/1.parquet], [/testdata/tpch/data/orders/2.parquet], [/testdata/tpch/data/orders/3.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=[] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/customer/1.parquet], [/testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet]]}, projection=[c_custkey, c_nationkey], file_type=parquet - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.parquet]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount], file_type=parquet - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/nation/1.parquet], [/testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet]]}, projection=[n_nationkey, n_name, n_regionkey], file_type=parquet + ┌───── Stage 1 Tasks: t0:[p0,p1,p2] + │ RepartitionExec: partitioning=Hash([n_name@0], 3), input_partitions=3 + │ 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 + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/region/1.parquet], [/testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.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)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@4, n_regionkey@5] + │ CoalescePartitionsExec + │ ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, s_nationkey@0 as s_nationkey] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1), (s_nationkey@1, c_nationkey@0)], projection=[s_nationkey@1, l_extendedprice@4, l_discount@5] + │ CoalescePartitionsExec + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/supplier/1.parquet], [/testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet]]}, projection=[s_suppkey, s_nationkey], file_type=parquet + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@1, l_orderkey@0)], projection=[c_nationkey@0, l_suppkey@3, l_extendedprice@4, l_discount@5] + │ CoalescePartitionsExec + │ ProjectionExec: expr=[c_nationkey@1 as c_nationkey, o_orderkey@0 as o_orderkey] + │ 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 + │ 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] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/orders/1.parquet], [/testdata/tpch/data/orders/2.parquet], [/testdata/tpch/data/orders/3.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=[] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/customer/1.parquet], [/testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet]]}, projection=[c_custkey, c_nationkey], file_type=parquet + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.parquet]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount], file_type=parquet + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/nation/1.parquet], [/testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet]]}, projection=[n_nationkey, n_name, n_regionkey], file_type=parquet └────────────────────────────────────────────────── "); Ok(()) @@ -219,14 +219,14 @@ mod tests { async fn test_tpch_6() -> Result<(), Box> { let plan = test_tpch_query(6).await?; assert_snapshot!(plan, @r" - ┌───── Stage 1 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:1 ] ProjectionExec: expr=[sum(lineitem.l_extendedprice * lineitem.l_discount)@0 as revenue] - │partitions [out:1 <-- in:1 ] AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.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=[] + ┌───── Stage 1 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 + │ 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] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.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=[] └────────────────────────────────────────────────── "); Ok(()) @@ -236,47 +236,47 @@ mod tests { async fn test_tpch_7() -> Result<(), Box> { let plan = test_tpch_query(7).await?; assert_snapshot!(plan, @r" - ┌───── Stage 2 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:3 ] SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST] - │partitions [out:3 <-- in:3 ] SortExec: expr=[supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST], preserve_partitioning=[true] - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 <-- in:3 ] 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)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 ] ArrowFlightReadExec: Stage 1 + ┌───── Stage 2 Tasks: t0:[p0] + │ SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST] + │ SortExec: expr=[supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year, sum(shipping.volume)@3 as revenue] + │ AggregateExec: mode=FinalPartitioned, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ ArrowFlightReadExec input_stage=1, input_partitions=3, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 1 Task: partitions: 0..2,unassigned] - │partitions [out:3 <-- in:3 ] RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 3), input_partitions=3 - │partitions [out:3 <-- in:3 ] 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)] - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] 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] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/nation/1.parquet], [/testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.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)] - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] 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] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/nation/1.parquet], [/testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.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)] - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@4)], projection=[c_nationkey@1, s_nationkey@2, l_extendedprice@3, l_discount@4, l_shipdate@5] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/customer/1.parquet], [/testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet]]}, projection=[c_custkey, c_nationkey], file_type=parquet - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, 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] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] 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] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/supplier/1.parquet], [/testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet]]}, projection=[s_suppkey, s_nationkey], file_type=parquet - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.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, 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=[] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/orders/1.parquet], [/testdata/tpch/data/orders/2.parquet], [/testdata/tpch/data/orders/3.parquet]]}, projection=[o_orderkey, o_custkey], file_type=parquet + ┌───── Stage 1 Tasks: t0:[p0,p1,p2] + │ RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 3), input_partitions=3 + │ 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 + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/nation/1.parquet], [/testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.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)] + │ 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 + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/nation/1.parquet], [/testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.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)] + │ 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=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@4)], projection=[c_nationkey@1, s_nationkey@2, l_extendedprice@3, l_discount@4, l_shipdate@5] + │ CoalescePartitionsExec + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/customer/1.parquet], [/testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet]]}, projection=[c_custkey, c_nationkey], file_type=parquet + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, 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] + │ CoalescePartitionsExec + │ 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] + │ CoalescePartitionsExec + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/supplier/1.parquet], [/testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet]]}, projection=[s_suppkey, s_nationkey], file_type=parquet + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.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, 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=[] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/orders/1.parquet], [/testdata/tpch/data/orders/2.parquet], [/testdata/tpch/data/orders/3.parquet]]}, projection=[o_orderkey, o_custkey], file_type=parquet └────────────────────────────────────────────────── "); Ok(()) @@ -286,57 +286,57 @@ mod tests { async fn test_tpch_8() -> Result<(), Box> { let plan = test_tpch_query(8).await?; assert_snapshot!(plan, @r#" - ┌───── Stage 2 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:3 ] SortPreservingMergeExec: [o_year@0 ASC NULLS LAST] - │partitions [out:3 <-- in:3 ] SortExec: expr=[o_year@0 ASC NULLS LAST], preserve_partitioning=[true] - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 <-- in:3 ] 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)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 ] ArrowFlightReadExec: Stage 1 + ┌───── Stage 2 Tasks: t0:[p0] + │ SortPreservingMergeExec: [o_year@0 ASC NULLS LAST] + │ SortExec: expr=[o_year@0 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[o_year@0 as o_year, sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END)@1 / sum(all_nations.volume)@2 as mkt_share] + │ AggregateExec: mode=FinalPartitioned, gby=[o_year@0 as o_year], aggr=[sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ ArrowFlightReadExec input_stage=1, input_partitions=3, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 1 Task: partitions: 0..2,unassigned] - │partitions [out:3 <-- in:3 ] RepartitionExec: partitioning=Hash([o_year@0], 3), input_partitions=3 - │partitions [out:3 <-- in:3 ] 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)] - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] 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] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/region/1.parquet], [/testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.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)] - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] 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] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/nation/1.parquet], [/testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet]]}, projection=[n_nationkey, n_name], file_type=parquet - │partitions [out:3 <-- in:3 ] ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, s_nationkey@3 as s_nationkey, o_orderdate@4 as o_orderdate, n_regionkey@0 as n_regionkey] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, c_nationkey@4)], projection=[n_regionkey@1, l_extendedprice@2, l_discount@3, s_nationkey@4, o_orderdate@5] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/nation/1.parquet], [/testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet]]}, projection=[n_nationkey, n_regionkey], file_type=parquet - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, 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] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_custkey@1, o_orderdate@2, l_extendedprice@4, l_discount@5, s_nationkey@6] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/orders/1.parquet], [/testdata/tpch/data/orders/2.parquet], [/testdata/tpch/data/orders/3.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=[] - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] 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] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/supplier/1.parquet], [/testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet]]}, projection=[s_suppkey, s_nationkey], file_type=parquet - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] 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] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: p_type@1 = ECONOMY ANODIZED STEEL, projection=[p_partkey@0] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/part/1.parquet], [/testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.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)] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.parquet]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount], file_type=parquet - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/customer/1.parquet], [/testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet]]}, projection=[c_custkey, c_nationkey], file_type=parquet + ┌───── Stage 1 Tasks: t0:[p0,p1,p2] + │ RepartitionExec: partitioning=Hash([o_year@0], 3), input_partitions=3 + │ 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 + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/region/1.parquet], [/testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.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)] + │ 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] + │ CoalescePartitionsExec + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/nation/1.parquet], [/testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet]]}, projection=[n_nationkey, n_name], file_type=parquet + │ ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, s_nationkey@3 as s_nationkey, o_orderdate@4 as o_orderdate, n_regionkey@0 as n_regionkey] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, c_nationkey@4)], projection=[n_regionkey@1, l_extendedprice@2, l_discount@3, s_nationkey@4, o_orderdate@5] + │ CoalescePartitionsExec + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/nation/1.parquet], [/testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet]]}, projection=[n_nationkey, n_regionkey], file_type=parquet + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, 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] + │ CoalescePartitionsExec + │ 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=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_custkey@1, o_orderdate@2, l_extendedprice@4, l_discount@5, s_nationkey@6] + │ CoalescePartitionsExec + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/orders/1.parquet], [/testdata/tpch/data/orders/2.parquet], [/testdata/tpch/data/orders/3.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=[] + │ 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 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5] + │ CoalescePartitionsExec + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/supplier/1.parquet], [/testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet]]}, projection=[s_suppkey, s_nationkey], file_type=parquet + │ 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 + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: p_type@1 = ECONOMY ANODIZED STEEL, projection=[p_partkey@0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/part/1.parquet], [/testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.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)] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.parquet]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount], file_type=parquet + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/customer/1.parquet], [/testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet]]}, projection=[c_custkey, c_nationkey], file_type=parquet └────────────────────────────────────────────────── "#); Ok(()) @@ -346,43 +346,43 @@ mod tests { async fn test_tpch_9() -> Result<(), Box> { let plan = test_tpch_query(9).await?; assert_snapshot!(plan, @r" - ┌───── Stage 2 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:3 ] SortPreservingMergeExec: [nation@0 ASC NULLS LAST, o_year@1 DESC] - │partitions [out:3 <-- in:3 ] SortExec: expr=[nation@0 ASC NULLS LAST, o_year@1 DESC], preserve_partitioning=[true] - │partitions [out:3 <-- in:3 ] ProjectionExec: expr=[nation@0 as nation, o_year@1 as o_year, sum(profit.amount)@2 as sum_profit] - │partitions [out:3 <-- in:3 ] AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 ] ArrowFlightReadExec: Stage 1 + ┌───── Stage 2 Tasks: t0:[p0] + │ SortPreservingMergeExec: [nation@0 ASC NULLS LAST, o_year@1 DESC] + │ SortExec: expr=[nation@0 ASC NULLS LAST, o_year@1 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[nation@0 as nation, o_year@1 as o_year, sum(profit.amount)@2 as sum_profit] + │ AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ ArrowFlightReadExec input_stage=1, input_partitions=3, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 1 Task: partitions: 0..2,unassigned] - │partitions [out:3 <-- in:3 ] RepartitionExec: partitioning=Hash([nation@0, o_year@1], 3), input_partitions=3 - │partitions [out:3 <-- in:3 ] AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[n_name@1, l_quantity@2, l_extendedprice@3, l_discount@4, ps_supplycost@6, o_orderdate@7] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/nation/1.parquet], [/testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet]]}, projection=[n_nationkey, n_name], file_type=parquet - │partitions [out:3 <-- in:3 ] ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@4 as s_nationkey, ps_supplycost@5 as ps_supplycost, o_orderdate@0 as o_orderdate] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@7] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/orders/1.parquet], [/testdata/tpch/data/orders/2.parquet], [/testdata/tpch/data/orders/3.parquet]]}, projection=[o_orderkey, o_orderdate], file_type=parquet - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, 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] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@2)], projection=[s_nationkey@1, l_orderkey@2, l_partkey@3, l_suppkey@4, l_quantity@5, l_extendedprice@6, l_discount@7] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/supplier/1.parquet], [/testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet]]}, projection=[s_suppkey, s_nationkey], file_type=parquet - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] 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] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: p_name@1 LIKE %green%, projection=[p_partkey@0] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/part/1.parquet], [/testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet]]}, projection=[p_partkey, p_name], file_type=parquet, predicate=p_name@1 LIKE %green% - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.parquet]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], file_type=parquet - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/partsupp/1.parquet], [/testdata/tpch/data/partsupp/2.parquet], [/testdata/tpch/data/partsupp/3.parquet]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet + ┌───── Stage 1 Tasks: t0:[p0,p1,p2] + │ RepartitionExec: partitioning=Hash([nation@0, o_year@1], 3), input_partitions=3 + │ 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] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[n_name@1, l_quantity@2, l_extendedprice@3, l_discount@4, ps_supplycost@6, o_orderdate@7] + │ CoalescePartitionsExec + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/nation/1.parquet], [/testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet]]}, projection=[n_nationkey, n_name], file_type=parquet + │ ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@4 as s_nationkey, ps_supplycost@5 as ps_supplycost, o_orderdate@0 as o_orderdate] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@7] + │ CoalescePartitionsExec + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/orders/1.parquet], [/testdata/tpch/data/orders/2.parquet], [/testdata/tpch/data/orders/3.parquet]]}, projection=[o_orderkey, o_orderdate], file_type=parquet + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, 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] + │ CoalescePartitionsExec + │ 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 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@2)], projection=[s_nationkey@1, l_orderkey@2, l_partkey@3, l_suppkey@4, l_quantity@5, l_extendedprice@6, l_discount@7] + │ CoalescePartitionsExec + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/supplier/1.parquet], [/testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet]]}, projection=[s_suppkey, s_nationkey], file_type=parquet + │ 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 + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: p_name@1 LIKE %green%, projection=[p_partkey@0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/part/1.parquet], [/testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet]]}, projection=[p_partkey, p_name], file_type=parquet, predicate=p_name@1 LIKE %green% + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.parquet]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], file_type=parquet + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/partsupp/1.parquet], [/testdata/tpch/data/partsupp/2.parquet], [/testdata/tpch/data/partsupp/3.parquet]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet └────────────────────────────────────────────────── "); Ok(()) @@ -392,36 +392,36 @@ mod tests { async fn test_tpch_10() -> Result<(), Box> { let plan = test_tpch_query(10).await?; assert_snapshot!(plan, @r" - ┌───── Stage 2 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:3 ] SortPreservingMergeExec: [revenue@2 DESC] - │partitions [out:3 <-- in:3 ] SortExec: expr=[revenue@2 DESC], preserve_partitioning=[true] - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 <-- in:3 ] 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)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 ] ArrowFlightReadExec: Stage 1 + ┌───── Stage 2 Tasks: t0:[p0] + │ SortPreservingMergeExec: [revenue@2 DESC] + │ SortExec: expr=[revenue@2 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@7 as revenue, c_acctbal@2 as c_acctbal, n_name@4 as n_name, c_address@5 as c_address, c_phone@3 as c_phone, c_comment@6 as c_comment] + │ AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@2 as c_acctbal, c_phone@3 as c_phone, n_name@4 as n_name, c_address@5 as c_address, c_comment@6 as c_comment], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ ArrowFlightReadExec input_stage=1, input_partitions=3, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 1 Task: partitions: 0..2,unassigned] - │partitions [out:3 <-- in:3 ] RepartitionExec: partitioning=Hash([c_custkey@0, c_name@1, c_acctbal@2, c_phone@3, n_name@4, c_address@5, c_comment@6], 3), input_partitions=3 - │partitions [out:3 <-- in:3 ] 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)] - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, c_nationkey@3)], projection=[n_name@1, c_custkey@2, c_name@3, c_address@4, c_phone@6, c_acctbal@7, c_comment@8, l_extendedprice@9, l_discount@10] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/nation/1.parquet], [/testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet]]}, projection=[n_nationkey, n_name], file_type=parquet - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@7, l_orderkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@9, l_discount@10] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] ProjectionExec: expr=[c_custkey@1 as c_custkey, c_name@2 as c_name, c_address@3 as c_address, c_nationkey@4 as c_nationkey, c_phone@5 as c_phone, c_acctbal@6 as c_acctbal, c_comment@7 as c_comment, o_orderkey@0 as o_orderkey] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] 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] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/orders/1.parquet], [/testdata/tpch/data/orders/2.parquet], [/testdata/tpch/data/orders/3.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=[] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/customer/1.parquet], [/testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet]]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment], file_type=parquet - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.parquet]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], file_type=parquet, predicate=l_returnflag@3 = R, 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,p2] + │ RepartitionExec: partitioning=Hash([c_custkey@0, c_name@1, c_acctbal@2, c_phone@3, n_name@4, c_address@5, c_comment@6], 3), input_partitions=3 + │ 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] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, c_nationkey@3)], projection=[n_name@1, c_custkey@2, c_name@3, c_address@4, c_phone@6, c_acctbal@7, c_comment@8, l_extendedprice@9, l_discount@10] + │ CoalescePartitionsExec + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/nation/1.parquet], [/testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet]]}, projection=[n_nationkey, n_name], file_type=parquet + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@7, l_orderkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@9, l_discount@10] + │ CoalescePartitionsExec + │ ProjectionExec: expr=[c_custkey@1 as c_custkey, c_name@2 as c_name, c_address@3 as c_address, c_nationkey@4 as c_nationkey, c_phone@5 as c_phone, c_acctbal@6 as c_acctbal, c_comment@7 as c_comment, o_orderkey@0 as o_orderkey] + │ 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 + │ 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] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/orders/1.parquet], [/testdata/tpch/data/orders/2.parquet], [/testdata/tpch/data/orders/3.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=[] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/customer/1.parquet], [/testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet]]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment], file_type=parquet + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.parquet]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], file_type=parquet, predicate=l_returnflag@3 = R, 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)] └────────────────────────────────────────────────── "); Ok(()) @@ -431,46 +431,46 @@ mod tests { async fn test_tpch_11() -> Result<(), Box> { let plan = test_tpch_query(11).await?; assert_snapshot!(plan, @r" - ┌───── Stage 2 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:3 ] SortPreservingMergeExec: [value@1 DESC] - │partitions [out:3 <-- in:3 ] SortExec: expr=[value@1 DESC], preserve_partitioning=[true] - │partitions [out:3 <-- in:3 ] ProjectionExec: expr=[ps_partkey@0 as ps_partkey, sum(partsupp.ps_supplycost * partsupp.ps_availqty)@1 as value] - │partitions [out:3 <-- in:1 ] NestedLoopJoinExec: join_type=Inner, filter=CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Decimal128(38, 15)) > sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)@1, projection=[ps_partkey@1, sum(partsupp.ps_supplycost * partsupp.ps_availqty)@2] - │partitions [out:1 <-- in:1 ] ProjectionExec: expr=[CAST(CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] - │partitions [out:1 <-- in:1 ] AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[ps_availqty@1, ps_supplycost@2] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/nation/1.parquet], [/testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.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)] - │partitions [out:3 <-- in:3 ] ProjectionExec: expr=[ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost, s_nationkey@0 as s_nationkey] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_nationkey@1, ps_availqty@3, ps_supplycost@4] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/supplier/1.parquet], [/testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet]]}, projection=[s_suppkey, s_nationkey], file_type=parquet - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/partsupp/1.parquet], [/testdata/tpch/data/partsupp/2.parquet], [/testdata/tpch/data/partsupp/3.parquet]]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], file_type=parquet - │partitions [out:3 <-- in:3 ] AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 ] ArrowFlightReadExec: Stage 1 + ┌───── Stage 2 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] + │ NestedLoopJoinExec: join_type=Inner, filter=CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Decimal128(38, 15)) > sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)@1, projection=[ps_partkey@1, sum(partsupp.ps_supplycost * partsupp.ps_availqty)@2] + │ ProjectionExec: expr=[CAST(CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] + │ AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] + │ CoalescePartitionsExec + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] + │ 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 + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/nation/1.parquet], [/testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.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)] + │ 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] + │ CoalescePartitionsExec + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/supplier/1.parquet], [/testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet]]}, projection=[s_suppkey, s_nationkey], file_type=parquet + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/partsupp/1.parquet], [/testdata/tpch/data/partsupp/2.parquet], [/testdata/tpch/data/partsupp/3.parquet]]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], file_type=parquet + │ AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ ArrowFlightReadExec input_stage=1, input_partitions=3, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 1 Task: partitions: 0..2,unassigned] - │partitions [out:3 <-- in:3 ] RepartitionExec: partitioning=Hash([ps_partkey@0], 3), input_partitions=3 - │partitions [out:3 <-- in:3 ] AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[ps_partkey@1, ps_availqty@2, ps_supplycost@3] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/nation/1.parquet], [/testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.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)] - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] 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] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/supplier/1.parquet], [/testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet]]}, projection=[s_suppkey, s_nationkey], file_type=parquet - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/partsupp/1.parquet], [/testdata/tpch/data/partsupp/2.parquet], [/testdata/tpch/data/partsupp/3.parquet]]}, projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost], file_type=parquet + ┌───── Stage 1 Tasks: t0:[p0,p1,p2] + │ RepartitionExec: partitioning=Hash([ps_partkey@0], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] + │ 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 + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/nation/1.parquet], [/testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.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)] + │ 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] + │ CoalescePartitionsExec + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/supplier/1.parquet], [/testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet]]}, projection=[s_suppkey, s_nationkey], file_type=parquet + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/partsupp/1.parquet], [/testdata/tpch/data/partsupp/2.parquet], [/testdata/tpch/data/partsupp/3.parquet]]}, projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost], file_type=parquet └────────────────────────────────────────────────── "); Ok(()) @@ -480,24 +480,24 @@ mod tests { async fn test_tpch_12() -> Result<(), Box> { let plan = test_tpch_query(12).await?; assert_snapshot!(plan, @r#" - ┌───── Stage 2 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:3 ] SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST] - │partitions [out:3 <-- in:3 ] SortExec: expr=[l_shipmode@0 ASC NULLS LAST], preserve_partitioning=[true] - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 <-- in:3 ] 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)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 ] ArrowFlightReadExec: Stage 1 + ┌───── Stage 2 Tasks: t0:[p0] + │ SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST] + │ SortExec: expr=[l_shipmode@0 ASC NULLS LAST], preserve_partitioning=[true] + │ ProjectionExec: expr=[l_shipmode@0 as l_shipmode, sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@1 as high_line_count, sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@2 as low_line_count] + │ AggregateExec: mode=FinalPartitioned, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ ArrowFlightReadExec input_stage=1, input_partitions=3, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 1 Task: partitions: 0..2,unassigned] - │partitions [out:3 <-- in:3 ] RepartitionExec: partitioning=Hash([l_shipmode@0], 3), input_partitions=3 - │partitions [out:3 <-- in:3 ] 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)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.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)] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/orders/1.parquet], [/testdata/tpch/data/orders/2.parquet], [/testdata/tpch/data/orders/3.parquet]]}, projection=[o_orderkey, o_orderpriority], file_type=parquet + ┌───── Stage 1 Tasks: t0:[p0,p1,p2] + │ RepartitionExec: partitioning=Hash([l_shipmode@0], 3), input_partitions=3 + │ 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=CollectLeft, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] + │ CoalescePartitionsExec + │ 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] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.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)] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/orders/1.parquet], [/testdata/tpch/data/orders/2.parquet], [/testdata/tpch/data/orders/3.parquet]]}, projection=[o_orderkey, o_orderpriority], file_type=parquet └────────────────────────────────────────────────── "#); Ok(()) @@ -507,33 +507,33 @@ mod tests { async fn test_tpch_13() -> Result<(), Box> { let plan = test_tpch_query(13).await?; assert_snapshot!(plan, @r" - ┌───── Stage 3 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:3 ] SortPreservingMergeExec: [custdist@1 DESC, c_count@0 DESC] - │partitions [out:3 <-- in:3 ] SortExec: expr=[custdist@1 DESC, c_count@0 DESC], preserve_partitioning=[true] - │partitions [out:3 <-- in:3 ] ProjectionExec: expr=[c_count@0 as c_count, count(Int64(1))@1 as custdist] - │partitions [out:3 <-- in:3 ] AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(Int64(1))] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 ] ArrowFlightReadExec: Stage 2 + ┌───── Stage 3 Tasks: t0:[p0] + │ SortPreservingMergeExec: [custdist@1 DESC, c_count@0 DESC] + │ SortExec: expr=[custdist@1 DESC, c_count@0 DESC], preserve_partitioning=[true] + │ ProjectionExec: expr=[c_count@0 as c_count, count(Int64(1))@1 as custdist] + │ AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(Int64(1))] + │ CoalesceBatchesExec: target_batch_size=8192 + │ ArrowFlightReadExec input_stage=2, input_partitions=3, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 2 Task: partitions: 0..2,unassigned] - │partitions [out:3 <-- in:3 ] RepartitionExec: partitioning=Hash([c_count@0], 3), input_partitions=3 - │partitions [out:3 <-- in:3 ] AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(Int64(1))] - │partitions [out:3 <-- in:3 ] ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count] - │partitions [out:3 <-- in:3 ] AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 ] ArrowFlightReadExec: Stage 1 + ┌───── Stage 2 Tasks: t0:[p0,p1,p2] + │ RepartitionExec: partitioning=Hash([c_count@0], 3), input_partitions=3 + │ 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 + │ ArrowFlightReadExec input_stage=1, input_partitions=3, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 1 Task: partitions: 0..2,unassigned] - │partitions [out:3 <-- in:3 ] RepartitionExec: partitioning=Hash([c_custkey@0], 3), input_partitions=3 - │partitions [out:3 <-- in:3 ] AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] - │partitions [out:3 <-- in:3 ] ProjectionExec: expr=[c_custkey@1 as c_custkey, o_orderkey@0 as o_orderkey] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Right, on=[(o_custkey@1, c_custkey@0)], projection=[o_orderkey@0, c_custkey@2] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/orders/1.parquet], [/testdata/tpch/data/orders/2.parquet], [/testdata/tpch/data/orders/3.parquet]]}, projection=[o_orderkey, o_custkey, o_comment], file_type=parquet, predicate=o_comment@2 NOT LIKE %special%requests% - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/customer/1.parquet], [/testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet]]}, projection=[c_custkey], file_type=parquet + ┌───── Stage 1 Tasks: t0:[p0,p1,p2] + │ RepartitionExec: partitioning=Hash([c_custkey@0], 3), input_partitions=3 + │ 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 + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/orders/1.parquet], [/testdata/tpch/data/orders/2.parquet], [/testdata/tpch/data/orders/3.parquet]]}, projection=[o_orderkey, o_custkey, o_comment], file_type=parquet, predicate=o_comment@2 NOT LIKE %special%requests% + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/customer/1.parquet], [/testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet]]}, projection=[c_custkey], file_type=parquet └────────────────────────────────────────────────── "); Ok(()) @@ -543,19 +543,19 @@ mod tests { async fn test_tpch_14() -> Result<(), Box> { let plan = test_tpch_query(14).await?; assert_snapshot!(plan, @r#" - ┌───── Stage 1 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:1 ] 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] - │partitions [out:1 <-- in:1 ] 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)] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] 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)] - │partitions [out:3 <-- in:3 ] ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as __common_expr_1, p_type@0 as p_type] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], projection=[p_type@1, l_extendedprice@3, l_discount@4] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/part/1.parquet], [/testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet]]}, projection=[p_partkey, p_type], file_type=parquet - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01, projection=[l_partkey@0, l_extendedprice@1, l_discount@2] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.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, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1995-09-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1995-10-01, required_guarantees=[] + ┌───── Stage 1 Tasks: t0:[p0] + │ 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 + │ 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=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], projection=[p_type@1, l_extendedprice@3, l_discount@4] + │ CoalescePartitionsExec + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/part/1.parquet], [/testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet]]}, projection=[p_partkey, p_type], file_type=parquet + │ 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] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.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, 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=[] └────────────────────────────────────────────────── "#); Ok(()) @@ -565,42 +565,42 @@ mod tests { async fn test_tpch_15() -> Result<(), Box> { let plan = test_tpch_query(15).await?; assert_snapshot!(plan, @r" - ┌───── Stage 3 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:3 ] SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST] - │partitions [out:3 <-- in:3 ] SortExec: expr=[s_suppkey@0 ASC NULLS LAST], preserve_partitioning=[true] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] 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] - │partitions [out:1 <-- in:1 ] AggregateExec: mode=Final, gby=[], aggr=[max(revenue0.total_revenue)] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] AggregateExec: mode=Partial, gby=[], aggr=[max(revenue0.total_revenue)] - │partitions [out:3 <-- in:3 ] ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] - │partitions [out:3 <-- in:3 ] AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 ] ArrowFlightReadExec: Stage 1 - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, supplier_no@0)], projection=[s_suppkey@0, s_name@1, s_address@2, s_phone@3, total_revenue@5] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/supplier/1.parquet], [/testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet]]}, projection=[s_suppkey, s_name, s_address, s_phone], file_type=parquet - │partitions [out:3 <-- in:3 ] ProjectionExec: expr=[l_suppkey@0 as supplier_no, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] - │partitions [out:3 <-- in:3 ] AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 ] ArrowFlightReadExec: Stage 2 + ┌───── Stage 3 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 + │ 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 + │ ArrowFlightReadExec input_stage=1, input_partitions=3, input_tasks=2 + │ 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 + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/supplier/1.parquet], [/testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet]]}, projection=[s_suppkey, s_name, s_address, s_phone], file_type=parquet + │ 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 + │ ArrowFlightReadExec input_stage=2, input_partitions=3, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 Task: partitions: 0,1,unassigned],Task: partitions: 2,unassigned] - │partitions [out:3 <-- in:2 ] RepartitionExec: partitioning=Hash([l_suppkey@0], 3), input_partitions=2 - │partitions [out:2 <-- in:3 ] PartitionIsolatorExec [providing upto 2 partitions] - │partitions [out:3 <-- in:3 ] AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.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 1 Tasks: t0:[p0,p1] t1:[p2] + │ RepartitionExec: partitioning=Hash([l_suppkey@0], 3), input_partitions=2 + │ PartitionIsolatorExec Tasks: t0:[p0,p1,__] t1:[__,__,p0] + │ 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] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.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 2 Task: partitions: 0,1,unassigned],Task: partitions: 2,unassigned] - │partitions [out:3 <-- in:2 ] RepartitionExec: partitioning=Hash([l_suppkey@0], 3), input_partitions=2 - │partitions [out:2 <-- in:3 ] PartitionIsolatorExec [providing upto 2 partitions] - │partitions [out:3 <-- in:3 ] AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.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 2 Tasks: t0:[p0,p1] t1:[p2] + │ RepartitionExec: partitioning=Hash([l_suppkey@0], 3), input_partitions=2 + │ PartitionIsolatorExec Tasks: t0:[p0,p1,__] t1:[__,__,p0] + │ 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] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.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=[] └────────────────────────────────────────────────── "); Ok(()) @@ -610,38 +610,38 @@ mod tests { async fn test_tpch_16() -> Result<(), Box> { let plan = test_tpch_query(16).await?; assert_snapshot!(plan, @r#" - ┌───── Stage 3 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:3 ] SortPreservingMergeExec: [supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST] - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 <-- in:3 ] 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)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 ] ArrowFlightReadExec: Stage 2 + ┌───── Stage 3 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] + │ 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 + │ ArrowFlightReadExec input_stage=2, input_partitions=3, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 2 Task: partitions: 0..2,unassigned] - │partitions [out:3 <-- in:3 ] RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2], 3), input_partitions=3 - │partitions [out:3 <-- in:3 ] 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)] - │partitions [out:3 <-- in:3 ] 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=[] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 ] ArrowFlightReadExec: Stage 1 + ┌───── Stage 2 Tasks: t0:[p0,p1,p2] + │ RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2], 3), input_partitions=3 + │ 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 + │ ArrowFlightReadExec input_stage=1, input_partitions=3, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 1 Task: partitions: 0..2,unassigned] - │partitions [out:3 <-- in:3 ] RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 3), input_partitions=3 - │partitions [out:3 <-- in:3 ] 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=[] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(s_suppkey@0, ps_suppkey@0)] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/supplier/1.parquet], [/testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet]]}, projection=[s_suppkey, s_comment], file_type=parquet, predicate=s_comment@1 LIKE %Customer%Complaints% - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] 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] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] 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: {} } }]) - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/part/1.parquet], [/testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.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)] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/partsupp/1.parquet], [/testdata/tpch/data/partsupp/2.parquet], [/testdata/tpch/data/partsupp/3.parquet]]}, projection=[ps_partkey, ps_suppkey], file_type=parquet + ┌───── Stage 1 Tasks: t0:[p0,p1,p2] + │ RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 3), input_partitions=3 + │ 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 + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/supplier/1.parquet], [/testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet]]}, projection=[s_suppkey, s_comment], file_type=parquet, predicate=s_comment@1 LIKE %Customer%Complaints% + │ 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 + │ 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: {} } }]) + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/part/1.parquet], [/testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.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)] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/partsupp/1.parquet], [/testdata/tpch/data/partsupp/2.parquet], [/testdata/tpch/data/partsupp/3.parquet]]}, projection=[ps_partkey, ps_suppkey], file_type=parquet └────────────────────────────────────────────────── "#); Ok(()) @@ -651,32 +651,32 @@ mod tests { async fn test_tpch_17() -> Result<(), Box> { let plan = test_tpch_query(17).await?; assert_snapshot!(plan, @r" - ┌───── Stage 2 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:1 ] ProjectionExec: expr=[CAST(sum(lineitem.l_extendedprice)@0 AS Float64) / 7 as avg_yearly] - │partitions [out:1 <-- in:1 ] AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice)] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * avg(lineitem.l_quantity)@1, projection=[l_extendedprice@1] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, p_partkey@0 as p_partkey] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], projection=[p_partkey@0, l_quantity@2, l_extendedprice@3] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/part/1.parquet], [/testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.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)] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.parquet]]}, projection=[l_partkey, l_quantity, l_extendedprice], file_type=parquet - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 <-- in:3 ] AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 ] ArrowFlightReadExec: Stage 1 + ┌───── Stage 2 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 + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * avg(lineitem.l_quantity)@1, projection=[l_extendedprice@1] + │ CoalescePartitionsExec + │ 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 + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/part/1.parquet], [/testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.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)] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.parquet]]}, projection=[l_partkey, l_quantity, l_extendedprice], file_type=parquet + │ 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 + │ ArrowFlightReadExec input_stage=1, input_partitions=3, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 Task: partitions: 0,1,unassigned],Task: partitions: 2,unassigned] - │partitions [out:3 <-- in:2 ] RepartitionExec: partitioning=Hash([l_partkey@0], 3), input_partitions=2 - │partitions [out:2 <-- in:3 ] PartitionIsolatorExec [providing upto 2 partitions] - │partitions [out:3 <-- in:3 ] AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.parquet]]}, projection=[l_partkey, l_quantity], file_type=parquet + ┌───── Stage 1 Tasks: t0:[p0,p1] t1:[p2] + │ RepartitionExec: partitioning=Hash([l_partkey@0], 3), input_partitions=2 + │ PartitionIsolatorExec Tasks: t0:[p0,p1,__] t1:[__,__,p0] + │ AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.parquet]]}, projection=[l_partkey, l_quantity], file_type=parquet └────────────────────────────────────────────────── "); Ok(()) @@ -686,39 +686,39 @@ mod tests { async fn test_tpch_18() -> Result<(), Box> { let plan = test_tpch_query(18).await?; assert_snapshot!(plan, @r" - ┌───── Stage 3 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:3 ] SortPreservingMergeExec: [o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST] - │partitions [out:3 <-- in:3 ] SortExec: expr=[o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST], preserve_partitioning=[true] - │partitions [out:3 <-- in:3 ] 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)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 ] ArrowFlightReadExec: Stage 2 + ┌───── Stage 3 Tasks: t0:[p0] + │ SortPreservingMergeExec: [o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST] + │ SortExec: expr=[o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST], preserve_partitioning=[true] + │ AggregateExec: mode=FinalPartitioned, gby=[c_name@0 as c_name, c_custkey@1 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@3 as o_orderdate, o_totalprice@4 as o_totalprice], aggr=[sum(lineitem.l_quantity)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ ArrowFlightReadExec input_stage=2, input_partitions=3, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 2 Task: partitions: 0..2,unassigned] - │partitions [out:3 <-- in:3 ] RepartitionExec: partitioning=Hash([c_name@0, c_custkey@1, o_orderkey@2, o_orderdate@3, o_totalprice@4], 3), input_partitions=3 - │partitions [out:3 <-- in:3 ] 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)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(l_orderkey@0, o_orderkey@2)] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0] - │partitions [out:3 <-- in:3 ] AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 ] ArrowFlightReadExec: Stage 1 - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@2, l_orderkey@0)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@3, o_orderdate@4, l_quantity@6] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@4, o_orderdate@5] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/customer/1.parquet], [/testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet]]}, projection=[c_custkey, c_name], file_type=parquet - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/orders/1.parquet], [/testdata/tpch/data/orders/2.parquet], [/testdata/tpch/data/orders/3.parquet]]}, projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate], file_type=parquet - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.parquet]]}, projection=[l_orderkey, l_quantity], file_type=parquet + ┌───── Stage 2 Tasks: t0:[p0,p1,p2] + │ RepartitionExec: partitioning=Hash([c_name@0, c_custkey@1, o_orderkey@2, o_orderdate@3, o_totalprice@4], 3), input_partitions=3 + │ 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 + │ 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 + │ ArrowFlightReadExec input_stage=1, input_partitions=3, input_tasks=2 + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@2, l_orderkey@0)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@3, o_orderdate@4, l_quantity@6] + │ CoalescePartitionsExec + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@4, o_orderdate@5] + │ CoalescePartitionsExec + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/customer/1.parquet], [/testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet]]}, projection=[c_custkey, c_name], file_type=parquet + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/orders/1.parquet], [/testdata/tpch/data/orders/2.parquet], [/testdata/tpch/data/orders/3.parquet]]}, projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate], file_type=parquet + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.parquet]]}, projection=[l_orderkey, l_quantity], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 1 Task: partitions: 0,1,unassigned],Task: partitions: 2,unassigned] - │partitions [out:3 <-- in:2 ] RepartitionExec: partitioning=Hash([l_orderkey@0], 3), input_partitions=2 - │partitions [out:2 <-- in:3 ] PartitionIsolatorExec [providing upto 2 partitions] - │partitions [out:3 <-- in:3 ] AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.parquet]]}, projection=[l_orderkey, l_quantity], file_type=parquet + ┌───── Stage 1 Tasks: t0:[p0,p1] t1:[p2] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 3), input_partitions=2 + │ PartitionIsolatorExec Tasks: t0:[p0,p1,__] t1:[__,__,p0] + │ AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.parquet]]}, projection=[l_orderkey, l_quantity], file_type=parquet └────────────────────────────────────────────────── "); Ok(()) @@ -728,20 +728,20 @@ mod tests { async fn test_tpch_19() -> Result<(), Box> { let plan = test_tpch_query(19).await?; assert_snapshot!(plan, @r#" - ┌───── Stage 1 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:1 ] ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@0 as revenue] - │partitions [out:1 <-- in:1 ] AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] 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] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] 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 - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/part/1.parquet], [/testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.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=[] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.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, pruning_predicate=(l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(100),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(1100),15,2 OR l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(1000),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(2000),15,2 OR l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(2000),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(3000),15,2) AND (l_shipmode_null_count@6 != row_count@2 AND l_shipmode_min@4 <= AIR AND AIR <= l_shipmode_max@5 OR l_shipmode_null_count@6 != row_count@2 AND l_shipmode_min@4 <= AIR REG AND AIR REG <= l_shipmode_max@5) AND l_shipinstruct_null_count@9 != row_count@2 AND l_shipinstruct_min@7 <= DELIVER IN PERSON AND DELIVER IN PERSON <= l_shipinstruct_max@8, required_guarantees=[l_shipinstruct in (DELIVER IN PERSON), l_shipmode in (AIR, AIR REG)] + ┌───── Stage 1 Tasks: t0:[p0] + │ 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 + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + │ 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 + │ 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 + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/part/1.parquet], [/testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.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=[] + │ 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={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.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, pruning_predicate=(l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(100),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(1100),15,2 OR l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(1000),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(2000),15,2 OR l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(2000),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(3000),15,2) AND (l_shipmode_null_count@6 != row_count@2 AND l_shipmode_min@4 <= AIR AND AIR <= l_shipmode_max@5 OR l_shipmode_null_count@6 != row_count@2 AND l_shipmode_min@4 <= AIR REG AND AIR REG <= l_shipmode_max@5) AND l_shipinstruct_null_count@9 != row_count@2 AND l_shipinstruct_min@7 <= DELIVER IN PERSON AND DELIVER IN PERSON <= l_shipinstruct_max@8, required_guarantees=[l_shipinstruct in (DELIVER IN PERSON), l_shipmode in (AIR, AIR REG)] └────────────────────────────────────────────────── "#); Ok(()) @@ -751,41 +751,41 @@ mod tests { async fn test_tpch_20() -> Result<(), Box> { let plan = test_tpch_query(20).await?; assert_snapshot!(plan, @r" - ┌───── Stage 2 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:3 ] SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] - │partitions [out:3 <-- in:3 ] SortExec: expr=[s_name@0 ASC NULLS LAST], preserve_partitioning=[true] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_name@1, s_address@2] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[s_suppkey@1, s_name@2, s_address@3] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/nation/1.parquet], [/testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.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)] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/supplier/1.parquet], [/testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet]]}, projection=[s_suppkey, s_name, s_address, s_nationkey], file_type=parquet - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] 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] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(p_partkey@0, ps_partkey@0)] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/part/1.parquet], [/testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.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=[] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/partsupp/1.parquet], [/testdata/tpch/data/partsupp/2.parquet], [/testdata/tpch/data/partsupp/3.parquet]]}, projection=[ps_partkey, ps_suppkey, ps_availqty], file_type=parquet - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 <-- in:3 ] AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 ] ArrowFlightReadExec: Stage 1 + ┌───── Stage 2 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 + │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_name@1, s_address@2] + │ CoalescePartitionsExec + │ 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 + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/nation/1.parquet], [/testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.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)] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/supplier/1.parquet], [/testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet]]}, projection=[s_suppkey, s_name, s_address, s_nationkey], file_type=parquet + │ 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] + │ CoalescePartitionsExec + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(p_partkey@0, ps_partkey@0)] + │ CoalescePartitionsExec + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/part/1.parquet], [/testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.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=[] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/partsupp/1.parquet], [/testdata/tpch/data/partsupp/2.parquet], [/testdata/tpch/data/partsupp/3.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 + │ ArrowFlightReadExec input_stage=1, input_partitions=3, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 Task: partitions: 0,1,unassigned],Task: partitions: 2,unassigned] - │partitions [out:3 <-- in:2 ] RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 3), input_partitions=2 - │partitions [out:2 <-- in:3 ] PartitionIsolatorExec [providing upto 2 partitions] - │partitions [out:3 <-- in:3 ] AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.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=[] + ┌───── Stage 1 Tasks: t0:[p0,p1] t1:[p2] + │ RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 3), input_partitions=2 + │ PartitionIsolatorExec Tasks: t0:[p0,p1,__] t1:[__,__,p0] + │ 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] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.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=[] └────────────────────────────────────────────────── "); Ok(()) @@ -795,46 +795,46 @@ mod tests { async fn test_tpch_21() -> Result<(), Box> { let plan = test_tpch_query(21).await?; assert_snapshot!(plan, @r" - ┌───── Stage 2 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:3 ] SortPreservingMergeExec: [numwait@1 DESC, s_name@0 ASC NULLS LAST] - │partitions [out:3 <-- in:3 ] SortExec: expr=[numwait@1 DESC, s_name@0 ASC NULLS LAST], preserve_partitioning=[true] - │partitions [out:3 <-- in:3 ] ProjectionExec: expr=[s_name@0 as s_name, count(Int64(1))@1 as numwait] - │partitions [out:3 <-- in:3 ] AggregateExec: mode=FinalPartitioned, gby=[s_name@0 as s_name], aggr=[count(Int64(1))] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 ] ArrowFlightReadExec: Stage 1 + ┌───── Stage 2 Tasks: t0:[p0] + │ SortPreservingMergeExec: [numwait@1 DESC, s_name@0 ASC NULLS LAST] + │ 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 + │ ArrowFlightReadExec input_stage=1, input_partitions=3, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 1 Task: partitions: 0..2,unassigned] - │partitions [out:3 <-- in:3 ] RepartitionExec: partitioning=Hash([s_name@0], 3), input_partitions=3 - │partitions [out:3 <-- in:3 ] AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[count(Int64(1))] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, projection=[s_name@0] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@1)], projection=[s_name@1, l_orderkey@3, l_suppkey@4] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/nation/1.parquet], [/testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.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)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] 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] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/orders/1.parquet], [/testdata/tpch/data/orders/2.parquet], [/testdata/tpch/data/orders/3.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)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] 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] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/supplier/1.parquet], [/testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet]]}, projection=[s_suppkey, s_name, s_nationkey], file_type=parquet - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.parquet]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@3 > l_commitdate@2 - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.parquet]]}, projection=[l_orderkey, l_suppkey], file_type=parquet - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.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,p2] + │ RepartitionExec: partitioning=Hash([s_name@0], 3), input_partitions=3 + │ AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[count(Int64(1))] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, projection=[s_name@0] + │ CoalescePartitionsExec + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 + │ CoalescePartitionsExec + │ 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 + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/nation/1.parquet], [/testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.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)] + │ 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 + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/orders/1.parquet], [/testdata/tpch/data/orders/2.parquet], [/testdata/tpch/data/orders/3.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)] + │ 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 + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/supplier/1.parquet], [/testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet]]}, projection=[s_suppkey, s_name, s_nationkey], file_type=parquet + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.parquet]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@3 > l_commitdate@2 + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.parquet]]}, projection=[l_orderkey, l_suppkey], file_type=parquet + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/lineitem/1.parquet], [/testdata/tpch/data/lineitem/2.parquet], [/testdata/tpch/data/lineitem/3.parquet]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@3 > l_commitdate@2 └────────────────────────────────────────────────── "); Ok(()) @@ -844,32 +844,32 @@ mod tests { async fn test_tpch_22() -> Result<(), Box> { let plan = test_tpch_query(22).await?; assert_snapshot!(plan, @r#" - ┌───── Stage 2 Task: partitions: 0,unassigned] - │partitions [out:1 <-- in:3 ] SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST] - │partitions [out:3 <-- in:3 ] SortExec: expr=[cntrycode@0 ASC NULLS LAST], preserve_partitioning=[true] - │partitions [out:3 <-- in:3 ] ProjectionExec: expr=[cntrycode@0 as cntrycode, count(Int64(1))@1 as numcust, sum(custsale.c_acctbal)@2 as totacctbal] - │partitions [out:3 <-- in:3 ] AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 ] ArrowFlightReadExec: Stage 1 + ┌───── Stage 2 Tasks: t0:[p0] + │ SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST] + │ 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 + │ ArrowFlightReadExec input_stage=1, input_partitions=3, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 1 Task: partitions: 0..2,unassigned] - │partitions [out:3 <-- in:3 ] RepartitionExec: partitioning=Hash([cntrycode@0], 3), input_partitions=3 - │partitions [out:3 <-- in:3 ] AggregateExec: mode=Partial, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)] - │partitions [out:3 <-- in:3 ] ProjectionExec: expr=[substr(c_phone@0, 1, 2) as cntrycode, c_acctbal@1 as c_acctbal] - │partitions [out:3 <-- in:1 ] 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] - │partitions [out:1 <-- in:1 ] AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] 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] - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/customer/1.parquet], [/testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.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=[] - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:1 ] HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)], projection=[c_phone@1, c_acctbal@2] - │partitions [out:1 <-- in:3 ] CoalescePartitionsExec - │partitions [out:3 <-- in:3 ] CoalesceBatchesExec: target_batch_size=8192 - │partitions [out:3 <-- in:3 ] 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: {} } }]) - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/customer/1.parquet], [/testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.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: {} } }]) - │partitions [out:3 ] DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/orders/1.parquet], [/testdata/tpch/data/orders/2.parquet], [/testdata/tpch/data/orders/3.parquet]]}, projection=[o_custkey], file_type=parquet + ┌───── Stage 1 Tasks: t0:[p0,p1,p2] + │ RepartitionExec: partitioning=Hash([cntrycode@0], 3), input_partitions=3 + │ 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 + │ 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] + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/customer/1.parquet], [/testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.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=[] + │ 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 + │ 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: {} } }]) + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/customer/1.parquet], [/testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.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: {} } }]) + │ DataSourceExec: file_groups={3 groups: [[/testdata/tpch/data/orders/1.parquet], [/testdata/tpch/data/orders/2.parquet], [/testdata/tpch/data/orders/3.parquet]]}, projection=[o_custkey], file_type=parquet └────────────────────────────────────────────────── "#); Ok(())