diff --git a/benchmarks/cdk/bin/datafusion-bench.ts b/benchmarks/cdk/bin/datafusion-bench.ts index 0088ff21..73631305 100644 --- a/benchmarks/cdk/bin/datafusion-bench.ts +++ b/benchmarks/cdk/bin/datafusion-bench.ts @@ -15,6 +15,7 @@ async function main () { .option('-i, --iterations ', 'Number of iterations', '3') .option('--files-per-task ', 'Files per task', '4') .option('--cardinality-task-sf ', 'Cardinality task scale factor', '2') + .option('--shuffle-batch-size ', 'Shuffle batch coalescing size (number of rows)', '8192') .option('--query ', 'A specific query to run', undefined) .parse(process.argv); @@ -24,6 +25,7 @@ async function main () { const iterations = parseInt(options.iterations); const filesPerTask = parseInt(options.filesPerTask); const cardinalityTaskSf = parseInt(options.cardinalityTaskSf); + const shuffleBatchSize = parseInt(options.shuffleBatchSize); // Compare with previous results first const results: BenchmarkResults = { queries: [] }; @@ -33,7 +35,8 @@ async function main () { await query(createTablesSql(sf)) await query(` SET distributed.files_per_task=${filesPerTask}; - SET distributed.cardinality_task_count_factor=${cardinalityTaskSf} + SET distributed.cardinality_task_count_factor=${cardinalityTaskSf}; + SET distributed.shuffle_batch_size=${shuffleBatchSize} `) for (let id of IDS) { diff --git a/src/distributed_planner/distributed_config.rs b/src/distributed_planner/distributed_config.rs index 6d87a5a5..22294aec 100644 --- a/src/distributed_planner/distributed_config.rs +++ b/src/distributed_planner/distributed_config.rs @@ -24,6 +24,13 @@ extensions_options! { /// - If a node reduces the cardinality of the data, this factor will decrease. /// - In any other situation, this factor is left intact. pub cardinality_task_count_factor: f64, default = cardinality_task_count_factor_default() + /// Upon shuffling over the network, data streams need to be disassembled in a lot of output + /// partitions, which means the resulting streams might contain a lot of tiny record batches + /// to be sent over the wire. This parameter controls the batch size in number of rows for + /// the CoalesceBatchExec operator that is placed at the top of the stage for sending bigger + /// batches over the wire. + /// If set to 0, batch coalescing is disabled on network shuffle operations. + pub shuffle_batch_size: usize, default = 8192 /// Collection of [TaskEstimator]s that will be applied to leaf nodes in order to /// estimate how many tasks should be spawned for the [Stage] containing the leaf node. pub(crate) __private_task_estimator: CombinedTaskEstimator, default = CombinedTaskEstimator::default() diff --git a/src/distributed_planner/distributed_physical_optimizer_rule.rs b/src/distributed_planner/distributed_physical_optimizer_rule.rs index ecd9371b..970e9f05 100644 --- a/src/distributed_planner/distributed_physical_optimizer_rule.rs +++ b/src/distributed_planner/distributed_physical_optimizer_rule.rs @@ -12,6 +12,7 @@ use datafusion::common::tree_node::TreeNodeRecursion; use datafusion::error::DataFusionError; use datafusion::physical_expr::Partitioning; use datafusion::physical_plan::ExecutionPlanProperties; +use datafusion::physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion::physical_plan::execution_plan::CardinalityEffect; use datafusion::physical_plan::joins::{HashJoinExec, PartitionMode}; @@ -300,8 +301,26 @@ fn _apply_network_boundaries( return Ok(ctx); } let task_count = ctx.scale_task_count_and_swap()?; + // Network shuffles imply partitioning each data stream in a lot of different partitions, + // which means that each resulting stream might contain tiny batches. It's important to + // have decent sized batches here as this will ultimately be sent over the wire, and the + // penalty there for sending many tiny batches instead of few big ones is big. + // TODO: After https://github.com/apache/datafusion/issues/18782 is shipped, the batching + // will be integrated in RepartitionExec itself, so we will not need to add a + // CoalesceBatchesExec, we just need to tell RepartitionExec to output a + // `d_cfg.shuffle_batch_size` batch size. + // Tracked by https://github.com/datafusion-contrib/datafusion-distributed/issues/243 + if d_cfg.shuffle_batch_size > 0 { + ctx.plan = Arc::new(CoalesceBatchesExec::new(ctx.plan, d_cfg.shuffle_batch_size)); + } ctx.plan = Arc::new(NetworkShuffleExec::try_new(ctx.plan, task_count)?); return Ok(ctx); + } else if let Some(coalesce_batches) = ctx.plan.as_any().downcast_ref::() { + // If the batch coalescing is before the network boundary, remove it, as we don't + // want it there, we want it after, and the code that adds it lives just some lines above. + if coalesce_batches.input().is_network_boundary() { + ctx.plan = Arc::clone(coalesce_batches.input()); + } } // If this is a CoalescePartitionsExec, it means that the original plan is trying to @@ -548,15 +567,15 @@ mod tests { │ 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 - │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] t2:[p0..p7] - │ RepartitionExec: partitioning=Hash([RainToday@0], 8), input_partitions=4 - │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 - │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] - │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday], file_type=parquet + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([RainToday@0], 8), input_partitions=4 + │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 + │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] + │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday], file_type=parquet └────────────────────────────────────────────────── "); } @@ -580,15 +599,15 @@ mod tests { │ 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 - │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=2 + │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=2 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] - │ RepartitionExec: partitioning=Hash([RainToday@0], 8), input_partitions=4 - │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] - │ PartitionIsolatorExec: t0:[p0,p1,__] t1:[__,__,p0] - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday], file_type=parquet + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([RainToday@0], 8), input_partitions=4 + │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] + │ PartitionIsolatorExec: t0:[p0,p1,__] t1:[__,__,p0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday], file_type=parquet └────────────────────────────────────────────────── "); } @@ -634,15 +653,15 @@ mod tests { │ 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 - │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p3] t1:[p0..p3] t2:[p0..p3] - │ RepartitionExec: partitioning=Hash([RainToday@0], 4), input_partitions=4 - │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 - │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] - │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday], file_type=parquet + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([RainToday@0], 4), input_partitions=4 + │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 + │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] + │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday], file_type=parquet └────────────────────────────────────────────────── "); } @@ -691,15 +710,15 @@ mod tests { │ 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 - │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] t2:[p0..p7] - │ RepartitionExec: partitioning=Hash([RainToday@0], 8), input_partitions=4 - │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 - │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] - │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday], file_type=parquet + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([RainToday@0], 8), input_partitions=4 + │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 + │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] + │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday], file_type=parquet └────────────────────────────────────────────────── "); } @@ -760,32 +779,32 @@ mod tests { │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=2 │ ProjectionExec: expr=[avg(weather.MaxTemp)@1 as MaxTemp, RainTomorrow@0 as RainTomorrow] │ AggregateExec: mode=FinalPartitioned, gby=[RainTomorrow@0 as RainTomorrow], aggr=[avg(weather.MaxTemp)] - │ CoalesceBatchesExec: target_batch_size=8192 - │ [Stage 3] => NetworkShuffleExec: output_partitions=4, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=4, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] │ ProjectionExec: expr=[avg(weather.MinTemp)@1 as MinTemp, RainTomorrow@0 as RainTomorrow] │ AggregateExec: mode=FinalPartitioned, gby=[RainTomorrow@0 as RainTomorrow], aggr=[avg(weather.MinTemp)] - │ CoalesceBatchesExec: target_batch_size=8192 - │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] t2:[p0..p7] - │ RepartitionExec: partitioning=Hash([RainTomorrow@0], 8), input_partitions=4 - │ AggregateExec: mode=Partial, gby=[RainTomorrow@1 as RainTomorrow], aggr=[avg(weather.MinTemp)] - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: RainToday@1 = yes, projection=[MinTemp@0, RainTomorrow@2] - │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 - │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday, 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)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([RainTomorrow@0], 8), input_partitions=4 + │ AggregateExec: mode=Partial, gby=[RainTomorrow@1 as RainTomorrow], aggr=[avg(weather.MinTemp)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: RainToday@1 = yes, projection=[MinTemp@0, RainTomorrow@2] + │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday, RainTomorrow], file_type=parquet, predicate=RainToday@1 = yes, pruning_predicate=RainToday_null_count@2 != row_count@3 AND RainToday_min@0 <= yes AND yes <= RainToday_max@1, required_guarantees=[RainToday in (yes)] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p3] t1:[p0..p3] t2:[p0..p3] - │ RepartitionExec: partitioning=Hash([RainTomorrow@0], 4), input_partitions=4 - │ AggregateExec: mode=Partial, gby=[RainTomorrow@1 as RainTomorrow], aggr=[avg(weather.MaxTemp)] - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: RainToday@1 = no, projection=[MaxTemp@0, RainTomorrow@2] - │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 - │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday, 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)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([RainTomorrow@0], 4), input_partitions=4 + │ AggregateExec: mode=Partial, gby=[RainTomorrow@1 as RainTomorrow], aggr=[avg(weather.MaxTemp)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: RainToday@1 = no, projection=[MaxTemp@0, RainTomorrow@2] + │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 + │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday, 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)] └────────────────────────────────────────────────── "); } @@ -828,15 +847,15 @@ mod tests { └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[] - │ CoalesceBatchesExec: target_batch_size=8192 - │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] t2:[p0..p7] - │ RepartitionExec: partitioning=Hash([RainToday@0, WindGustDir@1], 8), input_partitions=4 - │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 - │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[] - │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday, WindGustDir], file_type=parquet + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([RainToday@0, WindGustDir@1], 8), input_partitions=4 + │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 + │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[] + │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday, WindGustDir], file_type=parquet └────────────────────────────────────────────────── "); } diff --git a/src/execution_plans/network_shuffle.rs b/src/execution_plans/network_shuffle.rs index 1e2f3138..fe0c34a7 100644 --- a/src/execution_plans/network_shuffle.rs +++ b/src/execution_plans/network_shuffle.rs @@ -13,6 +13,7 @@ use arrow_flight::decode::FlightRecordBatchStream; use arrow_flight::error::FlightError; use bytes::Bytes; use dashmap::DashMap; +use datafusion::common::tree_node::{Transformed, TreeNode, TreeNodeRecursion}; use datafusion::common::{exec_err, internal_datafusion_err, plan_err}; use datafusion::error::DataFusionError; use datafusion::execution::{SendableRecordBatchStream, TaskContext}; @@ -172,18 +173,29 @@ impl NetworkBoundary for NetworkShuffleExec { return plan_err!("cannot only return wrapped child if on Pending state"); }; - // TODO: Avoid downcasting once https://github.com/apache/datafusion/pull/17990 is shipped. - let Some(r_exe) = pending.input.as_any().downcast_ref::() else { - return plan_err!("NetworkShuffleExec.input must always be RepartitionExec"); - }; - - let next_stage_plan = Arc::new(RepartitionExec::try_new( - require_one_child(r_exe.children())?, - scale_partitioning(r_exe.partitioning(), |p| p * n_tasks), - )?); + let transformed = Arc::clone(&pending.input).transform_down(|plan| { + if let Some(r_exe) = plan.as_any().downcast_ref::() { + // Scale the input RepartitionExec to account for all the tasks to which it will + // need to fan data out. + let scaled = Arc::new(RepartitionExec::try_new( + require_one_child(r_exe.children())?, + scale_partitioning(r_exe.partitioning(), |p| p * n_tasks), + )?); + Ok(Transformed::new(scaled, true, TreeNodeRecursion::Stop)) + } else if matches!(plan.output_partitioning(), Partitioning::Hash(_, _)) { + // This might be a passthrough node, like a CoalesceBatchesExec or something like that. + // This is fine, we can let the node be here. + Ok(Transformed::no(plan)) + } else { + return plan_err!( + "NetworkShuffleExec input must be hash partitioned, but {} is not", + plan.name() + ); + } + })?; Ok(InputStageInfo { - plan: next_stage_plan, + plan: transformed.data, task_count: pending.input_tasks, }) } diff --git a/src/test_utils/insta.rs b/src/test_utils/insta.rs index 110cd016..972f850f 100644 --- a/src/test_utils/insta.rs +++ b/src/test_utils/insta.rs @@ -164,6 +164,10 @@ pub fn settings() -> insta::Settings { settings.add_filter(r"input_batches=\d+", "input_batches="); settings.add_filter(r"input_rows=\d+", "input_rows="); settings.add_filter(r"output_batches=\d+", "output_batches="); + settings.add_filter( + r"output_bytes=\d+.\d [(B)|(KB)|(MB)]", + "output_bytes=", + ); settings.add_filter(r"build_mem_used=\d+", "build_mem_used="); settings.add_filter(r"build_time=[\d.]+[a-zA-Zµnms]+", "build_time="); settings.add_filter(r"join_time=[\d.]+[a-zA-Zµnms]+", "join_time="); diff --git a/tests/distributed_aggregation.rs b/tests/distributed_aggregation.rs index 4869bc51..fe36fe30 100644 --- a/tests/distributed_aggregation.rs +++ b/tests/distributed_aggregation.rs @@ -59,14 +59,14 @@ mod tests { │ 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 - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([RainToday@0], 6), input_partitions=1 - │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] - │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday], file_type=parquet + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([RainToday@0], 6), input_partitions=1 + │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] + │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday], file_type=parquet └────────────────────────────────────────────────── ", ); @@ -141,14 +141,14 @@ mod tests { ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday] │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] - │ CoalesceBatchesExec: target_batch_size=8192 - │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 + │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([RainToday@0], 6), input_partitions=1 - │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] - │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] - │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday], file_type=parquet + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([RainToday@0], 6), input_partitions=1 + │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] + │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] + │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday], file_type=parquet └────────────────────────────────────────────────── ", ); diff --git a/tests/tpch_explain_analyze.rs b/tests/tpch_explain_analyze.rs index 6395eb37..85aa41fa 100644 --- a/tests/tpch_explain_analyze.rs +++ b/tests/tpch_explain_analyze.rs @@ -21,24 +21,24 @@ mod tests { let plan = test_tpch_query(1).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] - │ SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST], metrics=[output_rows=4, elapsed_compute=, output_bytes=608.0 B] + │ SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST], metrics=[output_rows=4, elapsed_compute=, output_bytes=] │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=2, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] │ SortExec: expr=[l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=4, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ 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], metrics=[output_rows=4, elapsed_compute=] │ 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))], metrics=[output_rows=4, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=24, elapsed_compute=] - │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] + │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] - │ RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 12), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ 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))], metrics=[output_rows=24, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ 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], metrics=[output_rows=591856, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=591856, elapsed_compute=] - │ 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], metrics=[output_rows=591856, elapsed_compute=] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], file_type=parquet, predicate=l_shipdate@6 <= 1998-09-02, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@0 <= 1998-09-02, required_guarantees=[], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=24, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 12), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ 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))], metrics=[output_rows=24, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] + │ 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], metrics=[output_rows=591856, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=591856, elapsed_compute=] + │ 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], metrics=[output_rows=591856, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], file_type=parquet, predicate=l_shipdate@6 <= 1998-09-02, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@0 <= 1998-09-02, required_guarantees=[], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── "); Ok(()) @@ -49,7 +49,7 @@ mod tests { let plan = test_tpch_query(2).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] - │ SortPreservingMergeExec: [s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST], metrics=[output_rows=11264, elapsed_compute=, output_bytes=5.3 MB] + │ SortPreservingMergeExec: [s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST], metrics=[output_rows=11264, elapsed_compute=, output_bytes=B] │ [Stage 7] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] @@ -57,32 +57,31 @@ mod tests { │ 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], metrics=[output_rows=11264, elapsed_compute=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=11264, elapsed_compute=] │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@1), (ps_supplycost@7, min(partsupp.ps_supplycost)@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, n_name@8], metrics=[output_rows=11264, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16128, elapsed_compute=] - │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=11985, elapsed_compute=] - │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p23] - │ RepartitionExec: partitioning=Hash([p_partkey@0, ps_supplycost@7], 24), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16128, elapsed_compute=] - │ 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], metrics=[output_rows=16128, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=16, elapsed_compute=] - │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] - │ 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], metrics=[output_rows=4672, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=4672, elapsed_compute=] - │ 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], metrics=[output_rows=4672, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=400, elapsed_compute=] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/nation/1.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/10.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/12.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/13.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/15.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/16.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/3.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/4.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/6.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/7.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/8.parquet], ...]}, projection=[n_nationkey, n_name, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=400, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] - │ 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], metrics=[output_rows=292, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=292, elapsed_compute=] - │ 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], metrics=[output_rows=292, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/supplier/1.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/10.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/12.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/13.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/15.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/16.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/3.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/4.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/6.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/7.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=1000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=292, elapsed_compute=] - │ 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], metrics=[output_rows=292, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=73, elapsed_compute=] - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/partsupp/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ], metrics=[output_rows=80000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16128, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([p_partkey@0, ps_supplycost@7], 24), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16128, elapsed_compute=] + │ 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], metrics=[output_rows=16128, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=16, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ 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], metrics=[output_rows=4672, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=4672, elapsed_compute=] + │ 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], metrics=[output_rows=4672, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=400, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/nation/1.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/10.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/12.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/13.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/15.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/16.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/3.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/4.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/6.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/7.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/8.parquet], ...]}, projection=[n_nationkey, n_name, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=400, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ 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], metrics=[output_rows=292, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=292, elapsed_compute=] + │ 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], metrics=[output_rows=292, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/supplier/1.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/10.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/12.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/13.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/15.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/16.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/3.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/4.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/6.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/7.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=1000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=292, elapsed_compute=] + │ 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], metrics=[output_rows=292, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=73, elapsed_compute=] + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/partsupp/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ], metrics=[output_rows=80000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16, elapsed_compute=] @@ -97,30 +96,31 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/part/1.parquet, /testdata/tpch/explain_analyze_sf0.1/part/10.parquet, /testdata/tpch/explain_analyze_sf0.1/part/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/12.parquet, /testdata/tpch/explain_analyze_sf0.1/part/13.parquet, /testdata/tpch/explain_analyze_sf0.1/part/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/15.parquet, /testdata/tpch/explain_analyze_sf0.1/part/16.parquet, /testdata/tpch/explain_analyze_sf0.1/part/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/3.parquet, /testdata/tpch/explain_analyze_sf0.1/part/4.parquet, /testdata/tpch/explain_analyze_sf0.1/part/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/6.parquet, /testdata/tpch/explain_analyze_sf0.1/part/7.parquet, /testdata/tpch/explain_analyze_sf0.1/part/8.parquet], ...]}, projection=[p_partkey, p_mfgr, p_type, p_size], file_type=parquet, predicate=p_size@3 = 15 AND p_type@2 LIKE %BRASS, pruning_predicate=p_size_null_count@2 != row_count@3 AND p_size_min@0 <= 15 AND 15 <= p_size_max@1, required_guarantees=[p_size in (15)], metrics=[output_rows=20000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 6 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] - │ RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 24), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey], metrics=[output_rows=11985, elapsed_compute=] - │ AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)], metrics=[output_rows=11985, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=11985, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=11985, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 24), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey], metrics=[output_rows=11985, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)], metrics=[output_rows=11985, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p17] - │ RepartitionExec: partitioning=Hash([ps_partkey@0], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)], metrics=[output_rows=11985, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=4177920, elapsed_compute=] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@2)], projection=[ps_partkey@1, ps_supplycost@2], metrics=[output_rows=4177920, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=16, elapsed_compute=] - │ [Stage 4] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] - │ ProjectionExec: expr=[ps_partkey@1 as ps_partkey, ps_supplycost@2 as ps_supplycost, n_regionkey@0 as n_regionkey], metrics=[output_rows=1280000, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1280000, elapsed_compute=] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_regionkey@1, ps_partkey@2, ps_supplycost@3], metrics=[output_rows=1280000, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=400, elapsed_compute=] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/nation/1.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/10.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/12.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/13.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/15.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/16.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/3.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/4.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/6.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/7.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/8.parquet], ...]}, projection=[n_nationkey, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=400, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] - │ ProjectionExec: expr=[ps_partkey@1 as ps_partkey, ps_supplycost@2 as ps_supplycost, s_nationkey@0 as s_nationkey], metrics=[output_rows=80000, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=80000, elapsed_compute=] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@1)], projection=[s_nationkey@1, ps_partkey@2, ps_supplycost@4], metrics=[output_rows=80000, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/supplier/1.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/10.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/12.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/13.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/15.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/16.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/3.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/4.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/6.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/7.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=1000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/partsupp/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=80000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=11985, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([ps_partkey@0], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)], metrics=[output_rows=11985, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=4177920, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@2)], projection=[ps_partkey@1, ps_supplycost@2], metrics=[output_rows=4177920, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=16, elapsed_compute=] + │ [Stage 4] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ ProjectionExec: expr=[ps_partkey@1 as ps_partkey, ps_supplycost@2 as ps_supplycost, n_regionkey@0 as n_regionkey], metrics=[output_rows=1280000, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1280000, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_regionkey@1, ps_partkey@2, ps_supplycost@3], metrics=[output_rows=1280000, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=400, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/nation/1.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/10.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/12.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/13.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/15.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/16.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/3.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/4.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/6.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/7.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/8.parquet], ...]}, projection=[n_nationkey, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=400, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ ProjectionExec: expr=[ps_partkey@1 as ps_partkey, ps_supplycost@2 as ps_supplycost, s_nationkey@0 as s_nationkey], metrics=[output_rows=80000, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=80000, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@1)], projection=[s_nationkey@1, ps_partkey@2, ps_supplycost@4], metrics=[output_rows=80000, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/supplier/1.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/10.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/12.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/13.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/15.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/16.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/3.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/4.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/6.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/7.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=1000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/partsupp/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=80000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16, elapsed_compute=] @@ -137,32 +137,32 @@ mod tests { let plan = test_tpch_query(3).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] - │ SortPreservingMergeExec: [revenue@1 DESC, o_orderdate@2 ASC NULLS LAST], metrics=[output_rows=1216, elapsed_compute=, output_bytes=38.0 KB] + │ SortPreservingMergeExec: [revenue@1 DESC, o_orderdate@2 ASC NULLS LAST], metrics=[output_rows=1216, elapsed_compute=, output_bytes=B] │ [Stage 3] => NetworkCoalesceExec: output_partitions=18, input_tasks=3, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ SortExec: expr=[revenue@1 DESC, o_orderdate@2 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=1216, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ 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], metrics=[output_rows=1216, elapsed_compute=] │ 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)], metrics=[output_rows=1216, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1216, elapsed_compute=] - │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p17] - │ RepartitionExec: partitioning=Hash([l_orderkey@0, o_orderdate@1, o_shippriority@2], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ 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)], metrics=[output_rows=1216, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=3321, elapsed_compute=] - │ 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], metrics=[output_rows=3321, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=15224, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=15224, elapsed_compute=] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4], metrics=[output_rows=15224, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=3111, elapsed_compute=] - │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=72678, elapsed_compute=] - │ FilterExec: o_orderdate@2 < 1995-03-15, metrics=[output_rows=72678, elapsed_compute=] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/orders/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], file_type=parquet, predicate=o_orderdate@2 < 1995-03-15 AND DynamicFilter [ empty ], pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@0 < 1995-03-15, required_guarantees=[], metrics=[output_rows=150000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=324322, elapsed_compute=] - │ FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2], metrics=[output_rows=324322, elapsed_compute=] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 > 1995-03-15 AND DynamicFilter [ empty ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 > 1995-03-15, required_guarantees=[], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1216, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([l_orderkey@0, o_orderdate@1, o_shippriority@2], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ 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)], metrics=[output_rows=1216, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=3321, elapsed_compute=] + │ 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], metrics=[output_rows=3321, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=15224, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=15224, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4], metrics=[output_rows=15224, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=3111, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=72678, elapsed_compute=] + │ FilterExec: o_orderdate@2 < 1995-03-15, metrics=[output_rows=72678, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/orders/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], file_type=parquet, predicate=o_orderdate@2 < 1995-03-15 AND DynamicFilter [ empty ], pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@0 < 1995-03-15, required_guarantees=[], metrics=[output_rows=150000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=324322, elapsed_compute=] + │ FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2], metrics=[output_rows=324322, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 > 1995-03-15 AND DynamicFilter [ empty ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 > 1995-03-15, required_guarantees=[], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=3111, elapsed_compute=] @@ -179,26 +179,26 @@ mod tests { let plan = test_tpch_query(4).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] - │ SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST], metrics=[output_rows=5, elapsed_compute=, output_bytes=440.0 B] + │ SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST], metrics=[output_rows=5, elapsed_compute=, output_bytes=] │ [Stage 3] => NetworkCoalesceExec: output_partitions=18, input_tasks=3, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ SortExec: expr=[o_orderpriority@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=5, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ ProjectionExec: expr=[o_orderpriority@0 as o_orderpriority, count(Int64(1))@1 as order_count], metrics=[output_rows=5, elapsed_compute=] │ AggregateExec: mode=FinalPartitioned, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))], metrics=[output_rows=5, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=30, elapsed_compute=] - │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p17] - │ RepartitionExec: partitioning=Hash([o_orderpriority@0], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))], metrics=[output_rows=30, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5093, elapsed_compute=] - │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(l_orderkey@0, o_orderkey@0)], projection=[o_orderpriority@1], metrics=[output_rows=5093, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=379809, elapsed_compute=] - │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5552, elapsed_compute=] - │ FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2], metrics=[output_rows=5552, elapsed_compute=] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/orders/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderdate, o_orderpriority], file_type=parquet, predicate=o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1993-07-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 < 1993-10-01, required_guarantees=[], metrics=[output_rows=150000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=30, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([o_orderpriority@0], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))], metrics=[output_rows=30, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5093, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(l_orderkey@0, o_orderkey@0)], projection=[o_orderpriority@1], metrics=[output_rows=5093, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=379809, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5552, elapsed_compute=] + │ FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2], metrics=[output_rows=5552, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/orders/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderdate, o_orderpriority], file_type=parquet, predicate=o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1993-07-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 < 1993-10-01, required_guarantees=[], metrics=[output_rows=150000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=379809, elapsed_compute=] @@ -215,43 +215,43 @@ mod tests { let plan = test_tpch_query(5).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] - │ SortPreservingMergeExec: [revenue@1 DESC], metrics=[output_rows=5, elapsed_compute=, output_bytes=160.0 B] + │ SortPreservingMergeExec: [revenue@1 DESC], metrics=[output_rows=5, elapsed_compute=, output_bytes=] │ [Stage 4] => NetworkCoalesceExec: output_partitions=18, input_tasks=3, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ SortExec: expr=[revenue@1 DESC], preserve_partitioning=[true], metrics=[output_rows=5, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ ProjectionExec: expr=[n_name@0 as n_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as revenue], metrics=[output_rows=5, elapsed_compute=] │ AggregateExec: mode=FinalPartitioned, gby=[n_name@0 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=5, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=30, elapsed_compute=] - │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p17] - │ RepartitionExec: partitioning=Hash([n_name@0], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=30, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=221440, elapsed_compute=] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@3)], projection=[l_extendedprice@1, l_discount@2, n_name@3], metrics=[output_rows=221440, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=16, elapsed_compute=] - │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] - │ ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, n_name@0 as n_name, n_regionkey@1 as n_regionkey], metrics=[output_rows=59040, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=59040, elapsed_compute=] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_name@1, n_regionkey@2, l_extendedprice@3, l_discount@4], metrics=[output_rows=59040, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=400, elapsed_compute=] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/nation/1.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/10.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/12.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/13.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/15.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/16.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/3.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/4.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/6.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/7.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/8.parquet], ...]}, projection=[n_nationkey, n_name, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=400, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] - │ ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, s_nationkey@0 as s_nationkey], metrics=[output_rows=3690, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=3690, elapsed_compute=] - │ 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], metrics=[output_rows=3690, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/supplier/1.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/10.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/12.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/13.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/15.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/16.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/3.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/4.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/6.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/7.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=1000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=92293, elapsed_compute=] - │ 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], metrics=[output_rows=92293, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=22958, elapsed_compute=] - │ ProjectionExec: expr=[c_nationkey@1 as c_nationkey, o_orderkey@0 as o_orderkey], metrics=[output_rows=22958, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=22958, elapsed_compute=] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_custkey@1, c_custkey@0)], projection=[o_orderkey@0, c_nationkey@3], metrics=[output_rows=22958, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=22958, elapsed_compute=] - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/customer/1.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/10.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/12.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/13.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/15.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/16.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/3.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/4.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/6.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/7.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/8.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=15000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=30, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([n_name@0], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=30, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=221440, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@3)], projection=[l_extendedprice@1, l_discount@2, n_name@3], metrics=[output_rows=221440, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=16, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, n_name@0 as n_name, n_regionkey@1 as n_regionkey], metrics=[output_rows=59040, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=59040, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_name@1, n_regionkey@2, l_extendedprice@3, l_discount@4], metrics=[output_rows=59040, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=400, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/nation/1.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/10.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/12.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/13.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/15.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/16.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/3.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/4.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/6.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/7.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/8.parquet], ...]}, projection=[n_nationkey, n_name, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=400, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, s_nationkey@0 as s_nationkey], metrics=[output_rows=3690, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=3690, elapsed_compute=] + │ 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], metrics=[output_rows=3690, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/supplier/1.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/10.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/12.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/13.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/15.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/16.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/3.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/4.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/6.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/7.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=1000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=92293, elapsed_compute=] + │ 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], metrics=[output_rows=92293, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=22958, elapsed_compute=] + │ ProjectionExec: expr=[c_nationkey@1 as c_nationkey, o_orderkey@0 as o_orderkey], metrics=[output_rows=22958, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=22958, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_custkey@1, c_custkey@0)], projection=[o_orderkey@0, c_nationkey@3], metrics=[output_rows=22958, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=22958, elapsed_compute=] + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/customer/1.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/10.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/12.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/13.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/15.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/16.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/3.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/4.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/6.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/7.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/8.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=15000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16, elapsed_compute=] @@ -274,9 +274,9 @@ mod tests { let plan = test_tpch_query(6).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] - │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * lineitem.l_discount)@0 as revenue], metrics=[output_rows=1, elapsed_compute=, output_bytes=16.0 B] - │ AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)], metrics=[output_rows=1, elapsed_compute=, output_bytes=16.0 B] - │ CoalescePartitionsExec, metrics=[output_rows=8, elapsed_compute=, output_bytes=1024.0 B] + │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * lineitem.l_discount)@0 as revenue], metrics=[output_rows=1, elapsed_compute=, output_bytes=] + │ AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)], metrics=[output_rows=1, elapsed_compute=, output_bytes=] + │ CoalescePartitionsExec, metrics=[output_rows=8, elapsed_compute=, output_bytes=] │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] @@ -295,35 +295,33 @@ mod tests { let plan = test_tpch_query(7).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] - │ SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST], metrics=[output_rows=4, elapsed_compute=, output_bytes=208.0 B] + │ SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST], metrics=[output_rows=4, elapsed_compute=, output_bytes=] │ [Stage 8] => NetworkCoalesceExec: output_partitions=18, input_tasks=3, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ SortExec: expr=[supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=4, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ 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], metrics=[output_rows=4, elapsed_compute=] │ 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)], metrics=[output_rows=4, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=24, elapsed_compute=] - │ [Stage 7] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + │ [Stage 7] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p17] - │ RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ 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)], metrics=[output_rows=24, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ 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], metrics=[output_rows=164608, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=164608, elapsed_compute=] - │ 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], metrics=[output_rows=164608, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=32, elapsed_compute=] - │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] - │ 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], metrics=[output_rows=250096, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=250096, elapsed_compute=] - │ 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], metrics=[output_rows=250096, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=32, elapsed_compute=] - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] - │ 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], metrics=[output_rows=182762, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=182762, elapsed_compute=] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@4)], projection=[c_nationkey@1, s_nationkey@2, l_extendedprice@3, l_discount@4, l_shipdate@5], metrics=[output_rows=182762, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=15000, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=24, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ 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)], metrics=[output_rows=24, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] + │ 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], metrics=[output_rows=164608, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=164608, elapsed_compute=] + │ 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], metrics=[output_rows=164608, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=32, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ 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], metrics=[output_rows=250096, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=250096, elapsed_compute=] + │ 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], metrics=[output_rows=250096, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=32, elapsed_compute=] + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ 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], metrics=[output_rows=182762, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=182762, elapsed_compute=] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@4)], projection=[c_nationkey@1, s_nationkey@2, l_extendedprice@3, l_discount@4, l_shipdate@5], metrics=[output_rows=182762, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=182762, elapsed_compute=] │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] @@ -339,33 +337,35 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/nation/1.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/10.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/12.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/13.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/15.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/16.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/3.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/4.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/6.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/7.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = FRANCE OR n_name@1 = GERMANY, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= FRANCE AND FRANCE <= n_name_max@1 OR n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1, required_guarantees=[n_name in (FRANCE, GERMANY)], metrics=[output_rows=400, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_custkey@0], 6), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/customer/1.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/10.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/12.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/13.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/15.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/16.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/3.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/4.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/6.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/7.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/8.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=15000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=15000, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([c_custkey@0], 6), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/customer/1.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/10.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/12.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/13.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/15.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/16.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/3.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/4.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/6.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/7.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/8.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=15000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ RepartitionExec: partitioning=Hash([o_custkey@4], 6), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=182762, elapsed_compute=] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@1, o_orderkey@0)], projection=[s_nationkey@0, l_extendedprice@2, l_discount@3, l_shipdate@4, o_custkey@6], metrics=[output_rows=182762, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=182762, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=182762, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([o_custkey@4], 6), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=182762, elapsed_compute=] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@1, o_orderkey@0)], projection=[s_nationkey@0, l_extendedprice@2, l_discount@3, l_shipdate@4, o_custkey@6], metrics=[output_rows=182762, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=150000, elapsed_compute=] │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p23] - │ RepartitionExec: partitioning=Hash([l_orderkey@1], 24), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=182762, elapsed_compute=] - │ 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], metrics=[output_rows=182762, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/supplier/1.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/10.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/12.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/13.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/15.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/16.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/3.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/4.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/6.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/7.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=1000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=182762, elapsed_compute=] - │ FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31, metrics=[output_rows=182762, elapsed_compute=] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 AND DynamicFilter [ empty ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1995-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 <= 1996-12-31, required_guarantees=[], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=182762, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([l_orderkey@1], 24), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=182762, elapsed_compute=] + │ 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], metrics=[output_rows=182762, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/supplier/1.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/10.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/12.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/13.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/15.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/16.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/3.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/4.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/6.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/7.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=1000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=182762, elapsed_compute=] + │ FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31, metrics=[output_rows=182762, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 AND DynamicFilter [ empty ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1995-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 <= 1996-12-31, required_guarantees=[], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] - │ RepartitionExec: partitioning=Hash([o_orderkey@0], 24), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/orders/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ], metrics=[output_rows=150000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=150000, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 24), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/orders/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ], metrics=[output_rows=150000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── "); Ok(()) @@ -376,39 +376,37 @@ mod tests { let plan = test_tpch_query(8).await?; assert_snapshot!(plan, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] - │ SortPreservingMergeExec: [o_year@0 ASC NULLS LAST], metrics=[output_rows=2, elapsed_compute=, output_bytes=40.0 B] + │ SortPreservingMergeExec: [o_year@0 ASC NULLS LAST], metrics=[output_rows=2, elapsed_compute=, output_bytes=] │ [Stage 8] => NetworkCoalesceExec: output_partitions=18, input_tasks=3, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ SortExec: expr=[o_year@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=2, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ 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], metrics=[output_rows=2, elapsed_compute=] │ 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)], metrics=[output_rows=2, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=12, elapsed_compute=] - │ [Stage 7] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + │ [Stage 7] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p17] - │ RepartitionExec: partitioning=Hash([o_year@0], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ 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)], metrics=[output_rows=12, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ 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], metrics=[output_rows=1155072, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1155072, elapsed_compute=] - │ 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], metrics=[output_rows=1155072, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=16, elapsed_compute=] - │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] - │ 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], metrics=[output_rows=365824, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=365824, elapsed_compute=] - │ 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], metrics=[output_rows=365824, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=400, elapsed_compute=] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/nation/1.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/10.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/12.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/13.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/15.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/16.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/3.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/4.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/6.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/7.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, metrics=[output_rows=400, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] - │ 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], metrics=[output_rows=22864, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=22864, elapsed_compute=] - │ 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], metrics=[output_rows=22864, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=400, elapsed_compute=] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/nation/1.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/10.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/12.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/13.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/15.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/16.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/3.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/4.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/6.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/7.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/8.parquet], ...]}, projection=[n_nationkey, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=400, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1429, elapsed_compute=] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@3, c_custkey@0)], projection=[l_extendedprice@0, l_discount@1, s_nationkey@2, o_orderdate@4, c_nationkey@6], metrics=[output_rows=1429, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1429, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=12, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([o_year@0], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ 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)], metrics=[output_rows=12, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] + │ 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], metrics=[output_rows=1155072, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1155072, elapsed_compute=] + │ 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], metrics=[output_rows=1155072, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=16, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ 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], metrics=[output_rows=365824, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=365824, elapsed_compute=] + │ 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], metrics=[output_rows=365824, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=400, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/nation/1.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/10.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/12.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/13.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/15.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/16.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/3.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/4.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/6.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/7.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, metrics=[output_rows=400, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ 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], metrics=[output_rows=22864, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=22864, elapsed_compute=] + │ 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], metrics=[output_rows=22864, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=400, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/nation/1.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/10.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/12.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/13.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/15.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/16.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/3.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/4.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/6.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/7.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/8.parquet], ...]}, projection=[n_nationkey, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=400, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1429, elapsed_compute=] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@3, c_custkey@0)], projection=[l_extendedprice@0, l_discount@1, s_nationkey@2, o_orderdate@4, c_nationkey@6], metrics=[output_rows=1429, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=15000, elapsed_compute=] │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] @@ -418,34 +416,35 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/region/1.parquet, /testdata/tpch/explain_analyze_sf0.1/region/10.parquet, /testdata/tpch/explain_analyze_sf0.1/region/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/region/12.parquet, /testdata/tpch/explain_analyze_sf0.1/region/13.parquet, /testdata/tpch/explain_analyze_sf0.1/region/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/region/15.parquet, /testdata/tpch/explain_analyze_sf0.1/region/16.parquet, /testdata/tpch/explain_analyze_sf0.1/region/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/region/3.parquet, /testdata/tpch/explain_analyze_sf0.1/region/4.parquet, /testdata/tpch/explain_analyze_sf0.1/region/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/region/6.parquet, /testdata/tpch/explain_analyze_sf0.1/region/7.parquet, /testdata/tpch/explain_analyze_sf0.1/region/8.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = AMERICA, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= AMERICA AND AMERICA <= r_name_max@1, required_guarantees=[r_name in (AMERICA)], metrics=[output_rows=80, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] - │ RepartitionExec: partitioning=Hash([o_custkey@3], 6), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ 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], metrics=[output_rows=1429, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1429, elapsed_compute=] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_custkey@1, o_orderdate@2, l_extendedprice@4, l_discount@5, s_nationkey@6], metrics=[output_rows=1429, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=45624, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1429, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([o_custkey@3], 6), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ 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], metrics=[output_rows=1429, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1429, elapsed_compute=] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_custkey@1, o_orderdate@2, l_extendedprice@4, l_discount@5, s_nationkey@6], metrics=[output_rows=1429, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=4485, elapsed_compute=] │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] - │ RepartitionExec: partitioning=Hash([o_orderkey@0], 18), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=45624, elapsed_compute=] - │ FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31, metrics=[output_rows=45624, elapsed_compute=] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/orders/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31, pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1995-01-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 <= 1996-12-31, required_guarantees=[], metrics=[output_rows=150000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=45624, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 18), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=45624, elapsed_compute=] + │ FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31, metrics=[output_rows=45624, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/orders/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31, pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1995-01-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 <= 1996-12-31, required_guarantees=[], metrics=[output_rows=150000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p17] - │ RepartitionExec: partitioning=Hash([l_orderkey@0], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ 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], metrics=[output_rows=4485, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=4485, elapsed_compute=] - │ 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], metrics=[output_rows=4485, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/supplier/1.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/10.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/12.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/13.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/15.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/16.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/3.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/4.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/6.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/7.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=1000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=4485, elapsed_compute=] - │ 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], metrics=[output_rows=4485, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=147, elapsed_compute=] - │ [Stage 3] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=4485, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ 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], metrics=[output_rows=4485, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=4485, elapsed_compute=] + │ 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], metrics=[output_rows=4485, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/supplier/1.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/10.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/12.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/13.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/15.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/16.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/3.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/4.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/6.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/7.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=1000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=4485, elapsed_compute=] + │ 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], metrics=[output_rows=4485, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=147, elapsed_compute=] + │ [Stage 3] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=147, elapsed_compute=] @@ -454,9 +453,10 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/part/1.parquet, /testdata/tpch/explain_analyze_sf0.1/part/10.parquet, /testdata/tpch/explain_analyze_sf0.1/part/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/12.parquet, /testdata/tpch/explain_analyze_sf0.1/part/13.parquet, /testdata/tpch/explain_analyze_sf0.1/part/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/15.parquet, /testdata/tpch/explain_analyze_sf0.1/part/16.parquet, /testdata/tpch/explain_analyze_sf0.1/part/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/3.parquet, /testdata/tpch/explain_analyze_sf0.1/part/4.parquet, /testdata/tpch/explain_analyze_sf0.1/part/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/6.parquet, /testdata/tpch/explain_analyze_sf0.1/part/7.parquet, /testdata/tpch/explain_analyze_sf0.1/part/8.parquet], ...]}, projection=[p_partkey, p_type], file_type=parquet, predicate=p_type@1 = ECONOMY ANODIZED STEEL, pruning_predicate=p_type_null_count@2 != row_count@3 AND p_type_min@0 <= ECONOMY ANODIZED STEEL AND ECONOMY ANODIZED STEEL <= p_type_max@1, required_guarantees=[p_type in (ECONOMY ANODIZED STEEL)], metrics=[output_rows=20000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ RepartitionExec: partitioning=Hash([c_custkey@0], 6), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/customer/1.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/10.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/12.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/13.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/15.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/16.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/3.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/4.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/6.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/7.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/8.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ], metrics=[output_rows=15000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=15000, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([c_custkey@0], 6), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/customer/1.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/10.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/12.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/13.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/15.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/16.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/3.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/4.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/6.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/7.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/8.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ], metrics=[output_rows=15000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── "#); Ok(()) @@ -467,58 +467,57 @@ mod tests { let plan = test_tpch_query(9).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] - │ SortPreservingMergeExec: [nation@0 ASC NULLS LAST, o_year@1 DESC], metrics=[output_rows=175, elapsed_compute=, output_bytes=13.9 KB] + │ SortPreservingMergeExec: [nation@0 ASC NULLS LAST, o_year@1 DESC], metrics=[output_rows=175, elapsed_compute=, output_bytes=B] │ [Stage 7] => NetworkCoalesceExec: output_partitions=18, input_tasks=3, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ SortExec: expr=[nation@0 ASC NULLS LAST, o_year@1 DESC], preserve_partitioning=[true], metrics=[output_rows=175, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ ProjectionExec: expr=[nation@0 as nation, o_year@1 as o_year, sum(profit.amount)@2 as sum_profit], metrics=[output_rows=175, elapsed_compute=] │ AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)], metrics=[output_rows=175, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1050, elapsed_compute=] - │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 6 ── Tasks: t0:[p0..p17] - │ RepartitionExec: partitioning=Hash([nation@0, o_year@1], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)], metrics=[output_rows=1050, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ 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], metrics=[output_rows=514560, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=514560, elapsed_compute=] - │ 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], metrics=[output_rows=514560, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=400, elapsed_compute=] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/nation/1.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/10.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/12.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/13.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/15.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/16.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/3.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/4.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/6.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/7.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, metrics=[output_rows=400, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] - │ 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], metrics=[output_rows=32160, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=32160, elapsed_compute=] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@7], metrics=[output_rows=32160, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=150000, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1050, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([nation@0, o_year@1], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)], metrics=[output_rows=1050, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] + │ 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], metrics=[output_rows=514560, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=514560, elapsed_compute=] + │ 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], metrics=[output_rows=514560, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=400, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/nation/1.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/10.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/12.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/13.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/15.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/16.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/3.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/4.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/6.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/7.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, metrics=[output_rows=400, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ 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], metrics=[output_rows=32160, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=32160, elapsed_compute=] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@7], metrics=[output_rows=32160, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=32160, elapsed_compute=] │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ RepartitionExec: partitioning=Hash([o_orderkey@0], 6), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/orders/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderdate], file_type=parquet, metrics=[output_rows=150000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=150000, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 6), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/orders/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderdate], file_type=parquet, metrics=[output_rows=150000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ RepartitionExec: partitioning=Hash([l_orderkey@0], 6), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=32160, elapsed_compute=] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, ps_suppkey@1), (l_partkey@1, ps_partkey@0)], projection=[l_orderkey@0, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@9], metrics=[output_rows=32160, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=32160, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=32160, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 6), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=32160, elapsed_compute=] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, ps_suppkey@1), (l_partkey@1, ps_partkey@0)], projection=[l_orderkey@0, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@9], metrics=[output_rows=32160, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=80000, elapsed_compute=] │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p23] - │ RepartitionExec: partitioning=Hash([l_suppkey@2, l_partkey@1], 24), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ 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], metrics=[output_rows=32160, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=32160, elapsed_compute=] - │ 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], metrics=[output_rows=32160, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/supplier/1.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/10.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/12.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/13.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/15.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/16.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/3.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/4.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/6.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/7.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=1000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=32160, elapsed_compute=] - │ 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], metrics=[output_rows=32160, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=1075, elapsed_compute=] - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=32160, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([l_suppkey@2, l_partkey@1], 24), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ 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], metrics=[output_rows=32160, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=32160, elapsed_compute=] + │ 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], metrics=[output_rows=32160, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/supplier/1.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/10.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/12.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/13.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/15.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/16.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/3.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/4.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/6.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/7.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=1000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=32160, elapsed_compute=] + │ 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], metrics=[output_rows=32160, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=1075, elapsed_compute=] + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1075, elapsed_compute=] @@ -527,9 +526,10 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/part/1.parquet, /testdata/tpch/explain_analyze_sf0.1/part/10.parquet, /testdata/tpch/explain_analyze_sf0.1/part/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/12.parquet, /testdata/tpch/explain_analyze_sf0.1/part/13.parquet, /testdata/tpch/explain_analyze_sf0.1/part/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/15.parquet, /testdata/tpch/explain_analyze_sf0.1/part/16.parquet, /testdata/tpch/explain_analyze_sf0.1/part/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/3.parquet, /testdata/tpch/explain_analyze_sf0.1/part/4.parquet, /testdata/tpch/explain_analyze_sf0.1/part/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/6.parquet, /testdata/tpch/explain_analyze_sf0.1/part/7.parquet, /testdata/tpch/explain_analyze_sf0.1/part/8.parquet], ...]}, projection=[p_partkey, p_name], file_type=parquet, predicate=p_name@1 LIKE %green%, metrics=[output_rows=20000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] - │ RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 24), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/partsupp/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=80000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=80000, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 24), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/partsupp/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=80000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── "); Ok(()) @@ -540,36 +540,36 @@ mod tests { let plan = test_tpch_query(10).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] - │ SortPreservingMergeExec: [revenue@2 DESC], metrics=[output_rows=3767, elapsed_compute=, output_bytes=1382.4 KB] + │ SortPreservingMergeExec: [revenue@2 DESC], metrics=[output_rows=3767, elapsed_compute=, output_bytes=B] │ [Stage 3] => NetworkCoalesceExec: output_partitions=18, input_tasks=3, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ SortExec: expr=[revenue@2 DESC], preserve_partitioning=[true], metrics=[output_rows=3767, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ 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], metrics=[output_rows=3767, elapsed_compute=] │ 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)], metrics=[output_rows=3767, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=4648, elapsed_compute=] - │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p17] - │ RepartitionExec: partitioning=Hash([c_custkey@0, c_name@1, c_acctbal@2, c_phone@3, n_name@4, c_address@5, c_comment@6], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ 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)], metrics=[output_rows=4648, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ 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], metrics=[output_rows=183024, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=183024, elapsed_compute=] - │ 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], metrics=[output_rows=183024, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=400, elapsed_compute=] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/nation/1.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/10.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/12.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/13.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/15.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/16.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/3.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/4.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/6.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/7.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, metrics=[output_rows=400, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=11439, elapsed_compute=] - │ 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], metrics=[output_rows=11439, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=5677, elapsed_compute=] - │ 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], metrics=[output_rows=5677, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5677, elapsed_compute=] - │ 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], metrics=[output_rows=5677, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=5677, elapsed_compute=] - │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/customer/1.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/10.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/12.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/13.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/15.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/16.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/3.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/4.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/6.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/7.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/8.parquet], ...]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ], metrics=[output_rows=15000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=148301, elapsed_compute=] - │ FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2], metrics=[output_rows=148301, elapsed_compute=] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], file_type=parquet, predicate=l_returnflag@3 = R AND DynamicFilter [ empty ], pruning_predicate=l_returnflag_null_count@2 != row_count@3 AND l_returnflag_min@0 <= R AND R <= l_returnflag_max@1, required_guarantees=[l_returnflag in (R)], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=4648, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([c_custkey@0, c_name@1, c_acctbal@2, c_phone@3, n_name@4, c_address@5, c_comment@6], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ 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)], metrics=[output_rows=4648, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] + │ 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], metrics=[output_rows=183024, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=183024, elapsed_compute=] + │ 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], metrics=[output_rows=183024, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=400, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/nation/1.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/10.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/12.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/13.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/15.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/16.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/3.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/4.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/nation/6.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/7.parquet, /testdata/tpch/explain_analyze_sf0.1/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, metrics=[output_rows=400, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=11439, elapsed_compute=] + │ 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], metrics=[output_rows=11439, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=5677, elapsed_compute=] + │ 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], metrics=[output_rows=5677, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5677, elapsed_compute=] + │ 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], metrics=[output_rows=5677, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=5677, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/customer/1.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/10.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/12.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/13.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/15.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/16.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/3.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/4.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/6.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/7.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/8.parquet], ...]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ], metrics=[output_rows=15000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=148301, elapsed_compute=] + │ FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2], metrics=[output_rows=148301, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], file_type=parquet, predicate=l_returnflag@3 = R AND DynamicFilter [ empty ], pruning_predicate=l_returnflag_null_count@2 != row_count@3 AND l_returnflag_min@0 <= R AND R <= l_returnflag_max@1, required_guarantees=[l_returnflag in (R)], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5677, elapsed_compute=] @@ -586,39 +586,39 @@ mod tests { let plan = test_tpch_query(11).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] - │ SortPreservingMergeExec: [value@1 DESC], metrics=[output_rows=2541, elapsed_compute=, output_bytes=59.6 KB] - │ SortExec: expr=[value@1 DESC], preserve_partitioning=[true], metrics=[output_rows=2541, elapsed_compute=, output_bytes=0.0 B, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] - │ ProjectionExec: expr=[ps_partkey@1 as ps_partkey, sum(partsupp.ps_supplycost * partsupp.ps_availqty)@2 as value], metrics=[output_rows=2541, elapsed_compute=, output_bytes=1152.0 KB] - │ NestedLoopJoinExec: join_type=Inner, filter=join_proj_push_down_1@1 > sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)@0, projection=[sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)@0, ps_partkey@1, sum(partsupp.ps_supplycost * partsupp.ps_availqty)@2], metrics=[output_rows=2541, elapsed_compute=, output_bytes=1920.0 KB, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=, selectivity=68% (2541/3716)] - │ 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)], metrics=[output_rows=1, elapsed_compute=, output_bytes=64.0 B] - │ AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)], metrics=[output_rows=1, elapsed_compute=, output_bytes=16.0 B] - │ CoalescePartitionsExec, metrics=[output_rows=6, elapsed_compute=, output_bytes=96.0 B] - │ AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)], metrics=[output_rows=6, elapsed_compute=, output_bytes=96.0 B] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=64000, elapsed_compute=, output_bytes=2.5 MB] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[ps_availqty@1, ps_supplycost@2], metrics=[output_rows=64000, elapsed_compute=, output_bytes=1250.0 KB, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=16, elapsed_compute=, output_bytes=768.0 B] + │ SortPreservingMergeExec: [value@1 DESC], metrics=[output_rows=2541, elapsed_compute=, output_bytes=B] + │ SortExec: expr=[value@1 DESC], preserve_partitioning=[true], metrics=[output_rows=2541, elapsed_compute=, output_bytes=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] + │ ProjectionExec: expr=[ps_partkey@1 as ps_partkey, sum(partsupp.ps_supplycost * partsupp.ps_availqty)@2 as value], metrics=[output_rows=2541, elapsed_compute=, output_bytes=B] + │ NestedLoopJoinExec: join_type=Inner, filter=join_proj_push_down_1@1 > sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)@0, projection=[sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)@0, ps_partkey@1, sum(partsupp.ps_supplycost * partsupp.ps_availqty)@2], metrics=[output_rows=2541, elapsed_compute=, output_bytes=B, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=, selectivity=68% (2541/3716)] + │ 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)], metrics=[output_rows=1, elapsed_compute=, output_bytes=] + │ AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)], metrics=[output_rows=1, elapsed_compute=, output_bytes=] + │ CoalescePartitionsExec, metrics=[output_rows=6, elapsed_compute=, output_bytes=] + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)], metrics=[output_rows=6, elapsed_compute=, output_bytes=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=64000, elapsed_compute=, output_bytes=B] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[ps_availqty@1, ps_supplycost@2], metrics=[output_rows=64000, elapsed_compute=, output_bytes=B, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=16, elapsed_compute=, output_bytes=] │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] - │ ProjectionExec: expr=[ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost, s_nationkey@0 as s_nationkey], metrics=[output_rows=80000, elapsed_compute=, output_bytes=2.1 MB] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=80000, elapsed_compute=, output_bytes=2.1 MB] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_nationkey@1, ps_availqty@3, ps_supplycost@4], metrics=[output_rows=80000, elapsed_compute=, output_bytes=2.1 MB, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=, output_bytes=15.6 KB] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/supplier/1.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/10.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/12.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/13.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/15.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/16.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/3.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/4.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/6.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/7.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ s_nationkey@1 >= 7 AND s_nationkey@1 <= 7 ], pruning_predicate=s_nationkey_null_count@1 != row_count@2 AND s_nationkey_max@0 >= 7 AND s_nationkey_null_count@1 != row_count@2 AND s_nationkey_min@3 <= 7, required_guarantees=[], metrics=[output_rows=1000, elapsed_compute=, output_bytes=15.6 KB, files_ranges_pruned_statistics= total → X matched, row_groups_pruned_statistics= total → X matched, row_groups_pruned_bloom_filter= total → X matched, page_index_rows_pruned= total → X matched, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/partsupp/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/7.parquet:..], ...]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ ps_suppkey@0 >= 1 AND ps_suppkey@0 <= 1000 ], pruning_predicate=ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_max@0 >= 1 AND ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_min@3 <= 1000, required_guarantees=[], metrics=[output_rows=80000, elapsed_compute=, output_bytes=2.1 MB, files_ranges_pruned_statistics= total → X matched, row_groups_pruned_statistics= total → X matched, row_groups_pruned_bloom_filter= total → X matched, page_index_rows_pruned= total → X matched, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] - │ ProjectionExec: expr=[ps_partkey@0 as ps_partkey, sum(partsupp.ps_supplycost * partsupp.ps_availqty)@1 as sum(partsupp.ps_supplycost * partsupp.ps_availqty), CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@1 AS Decimal128(38, 15)) as join_proj_push_down_1], metrics=[output_rows=3716, elapsed_compute=, output_bytes=164.2 KB] - │ AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)], metrics=[output_rows=3716, elapsed_compute=, output_bytes=106.1 KB, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=3716, elapsed_compute=, output_bytes=2.2 MB] + │ ProjectionExec: expr=[ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost, s_nationkey@0 as s_nationkey], metrics=[output_rows=80000, elapsed_compute=, output_bytes=B] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=80000, elapsed_compute=, output_bytes=B] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_nationkey@1, ps_availqty@3, ps_supplycost@4], metrics=[output_rows=80000, elapsed_compute=, output_bytes=B, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=, output_bytes=B] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/supplier/1.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/10.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/12.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/13.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/15.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/16.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/3.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/4.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/6.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/7.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ s_nationkey@1 >= 7 AND s_nationkey@1 <= 7 ], pruning_predicate=s_nationkey_null_count@1 != row_count@2 AND s_nationkey_max@0 >= 7 AND s_nationkey_null_count@1 != row_count@2 AND s_nationkey_min@3 <= 7, required_guarantees=[], metrics=[output_rows=1000, elapsed_compute=, output_bytes=B, files_ranges_pruned_statistics= total → X matched, row_groups_pruned_statistics= total → X matched, row_groups_pruned_bloom_filter= total → X matched, page_index_rows_pruned= total → X matched, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/partsupp/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/7.parquet:..], ...]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ ps_suppkey@0 >= 1 AND ps_suppkey@0 <= 1000 ], pruning_predicate=ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_max@0 >= 1 AND ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_min@3 <= 1000, required_guarantees=[], metrics=[output_rows=80000, elapsed_compute=, output_bytes=B, files_ranges_pruned_statistics= total → X matched, row_groups_pruned_statistics= total → X matched, row_groups_pruned_bloom_filter= total → X matched, page_index_rows_pruned= total → X matched, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ ProjectionExec: expr=[ps_partkey@0 as ps_partkey, sum(partsupp.ps_supplycost * partsupp.ps_availqty)@1 as sum(partsupp.ps_supplycost * partsupp.ps_availqty), CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@1 AS Decimal128(38, 15)) as join_proj_push_down_1], metrics=[output_rows=3716, elapsed_compute=, output_bytes=B] + │ AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)], metrics=[output_rows=3716, elapsed_compute=, output_bytes=B, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=3716, elapsed_compute=, output_bytes=B] │ RepartitionExec: partitioning=Hash([ps_partkey@0], 6), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)], metrics=[output_rows=3716, elapsed_compute=, output_bytes=113.8 KB, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=, reduction_factor=5.8% (3716/64000)] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=64000, elapsed_compute=, output_bytes=3.5 MB] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[ps_partkey@1, ps_availqty@2, ps_supplycost@3], metrics=[output_rows=64000, elapsed_compute=, output_bytes=1750.0 KB, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=16, elapsed_compute=, output_bytes=768.0 B] + │ AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)], metrics=[output_rows=3716, elapsed_compute=, output_bytes=B, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=, reduction_factor=5.8% (3716/64000)] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=64000, elapsed_compute=, output_bytes=B] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[ps_partkey@1, ps_availqty@2, ps_supplycost@3], metrics=[output_rows=64000, elapsed_compute=, output_bytes=B, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=16, elapsed_compute=, output_bytes=] │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] - │ 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], metrics=[output_rows=80000, elapsed_compute=, output_bytes=2.7 MB] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=80000, elapsed_compute=, output_bytes=2.7 MB] - │ 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], metrics=[output_rows=80000, elapsed_compute=, output_bytes=2.7 MB, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=, output_bytes=15.6 KB] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/supplier/1.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/10.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/12.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/13.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/15.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/16.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/3.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/4.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/6.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/7.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ s_nationkey@1 >= 7 AND s_nationkey@1 <= 7 ], pruning_predicate=s_nationkey_null_count@1 != row_count@2 AND s_nationkey_max@0 >= 7 AND s_nationkey_null_count@1 != row_count@2 AND s_nationkey_min@3 <= 7, required_guarantees=[], metrics=[output_rows=1000, elapsed_compute=, output_bytes=15.6 KB, files_ranges_pruned_statistics= total → X matched, row_groups_pruned_statistics= total → X matched, row_groups_pruned_bloom_filter= total → X matched, page_index_rows_pruned= total → X matched, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/partsupp/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ ps_suppkey@1 >= 1 AND ps_suppkey@1 <= 1000 ], pruning_predicate=ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_max@0 >= 1 AND ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_min@3 <= 1000, required_guarantees=[], metrics=[output_rows=80000, elapsed_compute=, output_bytes=2.7 MB, files_ranges_pruned_statistics= total → X matched, row_groups_pruned_statistics= total → X matched, row_groups_pruned_bloom_filter= total → X matched, page_index_rows_pruned= total → X matched, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ 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], metrics=[output_rows=80000, elapsed_compute=, output_bytes=B] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=80000, elapsed_compute=, output_bytes=B] + │ 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], metrics=[output_rows=80000, elapsed_compute=, output_bytes=B, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=, output_bytes=B] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/supplier/1.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/10.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/12.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/13.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/15.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/16.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/3.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/4.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/6.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/7.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ s_nationkey@1 >= 7 AND s_nationkey@1 <= 7 ], pruning_predicate=s_nationkey_null_count@1 != row_count@2 AND s_nationkey_max@0 >= 7 AND s_nationkey_null_count@1 != row_count@2 AND s_nationkey_min@3 <= 7, required_guarantees=[], metrics=[output_rows=1000, elapsed_compute=, output_bytes=B, files_ranges_pruned_statistics= total → X matched, row_groups_pruned_statistics= total → X matched, row_groups_pruned_bloom_filter= total → X matched, page_index_rows_pruned= total → X matched, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/partsupp/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ ps_suppkey@1 >= 1 AND ps_suppkey@1 <= 1000 ], pruning_predicate=ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_max@0 >= 1 AND ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_min@3 <= 1000, required_guarantees=[], metrics=[output_rows=80000, elapsed_compute=, output_bytes=B, files_ranges_pruned_statistics= total → X matched, row_groups_pruned_statistics= total → X matched, row_groups_pruned_bloom_filter= total → X matched, page_index_rows_pruned= total → X matched, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16, elapsed_compute=] @@ -641,37 +641,37 @@ mod tests { let plan = test_tpch_query(12).await?; assert_snapshot!(plan, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] - │ SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST], metrics=[output_rows=2, elapsed_compute=, output_bytes=64.0 B] + │ SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST], metrics=[output_rows=2, elapsed_compute=, output_bytes=] │ [Stage 4] => NetworkCoalesceExec: output_partitions=18, input_tasks=3, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ SortExec: expr=[l_shipmode@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=2, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ 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], metrics=[output_rows=2, elapsed_compute=] │ 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)], metrics=[output_rows=2, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=36, elapsed_compute=] - │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] - │ RepartitionExec: partitioning=Hash([l_shipmode@0], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ 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)], metrics=[output_rows=36, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=3155, elapsed_compute=] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3], metrics=[output_rows=3155, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=3155, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=36, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([l_shipmode@0], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ 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)], metrics=[output_rows=36, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=3155, elapsed_compute=] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3], metrics=[output_rows=3155, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=150000, elapsed_compute=] │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] - │ RepartitionExec: partitioning=Hash([l_orderkey@0], 18), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=3155, elapsed_compute=] - │ 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], metrics=[output_rows=3155, elapsed_compute=] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], file_type=parquet, predicate=(l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, pruning_predicate=(l_shipmode_null_count@2 != row_count@3 AND l_shipmode_min@0 <= MAIL AND MAIL <= l_shipmode_max@1 OR l_shipmode_null_count@2 != row_count@3 AND l_shipmode_min@0 <= SHIP AND SHIP <= l_shipmode_max@1) AND l_receiptdate_null_count@5 != row_count@3 AND l_receiptdate_max@4 >= 1994-01-01 AND l_receiptdate_null_count@5 != row_count@3 AND l_receiptdate_min@6 < 1995-01-01, required_guarantees=[l_shipmode in (MAIL, SHIP)], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=3155, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 18), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=3155, elapsed_compute=] + │ 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], metrics=[output_rows=3155, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], file_type=parquet, predicate=(l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, pruning_predicate=(l_shipmode_null_count@2 != row_count@3 AND l_shipmode_min@0 <= MAIL AND MAIL <= l_shipmode_max@1 OR l_shipmode_null_count@2 != row_count@3 AND l_shipmode_min@0 <= SHIP AND SHIP <= l_shipmode_max@1) AND l_receiptdate_null_count@5 != row_count@3 AND l_receiptdate_max@4 >= 1994-01-01 AND l_receiptdate_null_count@5 != row_count@3 AND l_receiptdate_min@6 < 1995-01-01, required_guarantees=[l_shipmode in (MAIL, SHIP)], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] - │ RepartitionExec: partitioning=Hash([o_orderkey@0], 18), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/orders/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderpriority], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=150000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=150000, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 18), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/orders/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderpriority], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=150000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── "#); Ok(()) @@ -682,33 +682,33 @@ mod tests { let plan = test_tpch_query(13).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] - │ SortPreservingMergeExec: [custdist@1 DESC, c_count@0 DESC], metrics=[output_rows=37, elapsed_compute=, output_bytes=592.0 B] + │ SortPreservingMergeExec: [custdist@1 DESC, c_count@0 DESC], metrics=[output_rows=37, elapsed_compute=, output_bytes=] │ [Stage 4] => NetworkCoalesceExec: output_partitions=12, input_tasks=2, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] │ SortExec: expr=[custdist@1 DESC, c_count@0 DESC], preserve_partitioning=[true], metrics=[output_rows=37, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ ProjectionExec: expr=[c_count@0 as c_count, count(Int64(1))@1 as custdist], metrics=[output_rows=37, elapsed_compute=] │ AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(Int64(1))], metrics=[output_rows=37, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=568, elapsed_compute=] - │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] - │ RepartitionExec: partitioning=Hash([c_count@0], 12), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(Int64(1))], metrics=[output_rows=568, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count], metrics=[output_rows=15000, elapsed_compute=] - │ AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)], metrics=[output_rows=15000, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=15000, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=568, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([c_count@0], 12), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(Int64(1))], metrics=[output_rows=568, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] + │ ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count], metrics=[output_rows=15000, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)], metrics=[output_rows=15000, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p17] - │ RepartitionExec: partitioning=Hash([c_custkey@0], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)], metrics=[output_rows=15000, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ ProjectionExec: expr=[c_custkey@1 as c_custkey, o_orderkey@0 as o_orderkey], metrics=[output_rows=153318, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=153318, elapsed_compute=] - │ HashJoinExec: mode=CollectLeft, join_type=Right, on=[(o_custkey@1, c_custkey@0)], projection=[o_orderkey@0, c_custkey@2], metrics=[output_rows=153318, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=148318, elapsed_compute=] - │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/customer/1.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/10.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/12.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/13.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/15.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/16.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/3.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/4.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/6.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/7.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/8.parquet], ...]}, projection=[c_custkey], file_type=parquet, metrics=[output_rows=15000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=15000, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([c_custkey@0], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)], metrics=[output_rows=15000, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] + │ ProjectionExec: expr=[c_custkey@1 as c_custkey, o_orderkey@0 as o_orderkey], metrics=[output_rows=153318, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=153318, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Right, on=[(o_custkey@1, c_custkey@0)], projection=[o_orderkey@0, c_custkey@2], metrics=[output_rows=153318, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=148318, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/customer/1.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/10.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/12.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/13.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/15.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/16.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/3.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/4.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/6.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/7.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/8.parquet], ...]}, projection=[c_custkey], file_type=parquet, metrics=[output_rows=15000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=148318, elapsed_compute=] @@ -725,9 +725,9 @@ mod tests { let plan = test_tpch_query(14).await?; assert_snapshot!(plan, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] - │ ProjectionExec: expr=[100 * CAST(sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END)@0 AS Float64) / CAST(sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 AS Float64) as promo_revenue], metrics=[output_rows=1, elapsed_compute=, output_bytes=8.0 B] - │ 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)], metrics=[output_rows=1, elapsed_compute=, output_bytes=32.0 B] - │ CoalescePartitionsExec, metrics=[output_rows=24, elapsed_compute=, output_bytes=6.0 KB] + │ 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], metrics=[output_rows=1, elapsed_compute=, output_bytes=] + │ 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)], metrics=[output_rows=1, elapsed_compute=, output_bytes=] + │ CoalescePartitionsExec, metrics=[output_rows=24, elapsed_compute=, output_bytes=B] │ [Stage 3] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] @@ -735,22 +735,22 @@ mod tests { │ ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as __common_expr_1, p_type@0 as p_type], metrics=[output_rows=7630, elapsed_compute=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=7630, elapsed_compute=] │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], projection=[p_type@1, l_extendedprice@3, l_discount@4], metrics=[output_rows=7630, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=20000, elapsed_compute=] - │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=7630, elapsed_compute=] - │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] + │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] - │ RepartitionExec: partitioning=Hash([p_partkey@0], 24), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/part/1.parquet, /testdata/tpch/explain_analyze_sf0.1/part/10.parquet, /testdata/tpch/explain_analyze_sf0.1/part/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/12.parquet, /testdata/tpch/explain_analyze_sf0.1/part/13.parquet, /testdata/tpch/explain_analyze_sf0.1/part/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/15.parquet, /testdata/tpch/explain_analyze_sf0.1/part/16.parquet, /testdata/tpch/explain_analyze_sf0.1/part/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/3.parquet, /testdata/tpch/explain_analyze_sf0.1/part/4.parquet, /testdata/tpch/explain_analyze_sf0.1/part/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/6.parquet, /testdata/tpch/explain_analyze_sf0.1/part/7.parquet, /testdata/tpch/explain_analyze_sf0.1/part/8.parquet], ...]}, projection=[p_partkey, p_type], file_type=parquet, metrics=[output_rows=20000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=20000, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([p_partkey@0], 24), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/part/1.parquet, /testdata/tpch/explain_analyze_sf0.1/part/10.parquet, /testdata/tpch/explain_analyze_sf0.1/part/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/12.parquet, /testdata/tpch/explain_analyze_sf0.1/part/13.parquet, /testdata/tpch/explain_analyze_sf0.1/part/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/15.parquet, /testdata/tpch/explain_analyze_sf0.1/part/16.parquet, /testdata/tpch/explain_analyze_sf0.1/part/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/3.parquet, /testdata/tpch/explain_analyze_sf0.1/part/4.parquet, /testdata/tpch/explain_analyze_sf0.1/part/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/6.parquet, /testdata/tpch/explain_analyze_sf0.1/part/7.parquet, /testdata/tpch/explain_analyze_sf0.1/part/8.parquet], ...]}, projection=[p_partkey, p_type], file_type=parquet, metrics=[output_rows=20000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] - │ RepartitionExec: partitioning=Hash([l_partkey@0], 24), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=7630, elapsed_compute=] - │ FilterExec: l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01, projection=[l_partkey@0, l_extendedprice@1, l_discount@2], metrics=[output_rows=7630, elapsed_compute=] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01 AND DynamicFilter [ empty ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1995-09-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1995-10-01, required_guarantees=[], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=7630, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([l_partkey@0], 24), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=7630, elapsed_compute=] + │ FilterExec: l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01, projection=[l_partkey@0, l_extendedprice@1, l_discount@2], metrics=[output_rows=7630, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01 AND DynamicFilter [ empty ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1995-09-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1995-10-01, required_guarantees=[], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── "#); Ok(()) @@ -761,44 +761,44 @@ mod tests { let plan = test_tpch_query(15).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] - │ SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST], metrics=[output_rows=1, elapsed_compute=, output_bytes=24.1 KB] - │ SortExec: expr=[s_suppkey@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=1, elapsed_compute=, output_bytes=0.0 B, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1, elapsed_compute=, output_bytes=600.0 KB] - │ 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], metrics=[output_rows=1, elapsed_compute=, output_bytes=24.1 KB, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ AggregateExec: mode=Final, gby=[], aggr=[max(revenue0.total_revenue)], metrics=[output_rows=1, elapsed_compute=, output_bytes=16.0 B] - │ CoalescePartitionsExec, metrics=[output_rows=12, elapsed_compute=, output_bytes=1536.0 B] + │ SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST], metrics=[output_rows=1, elapsed_compute=, output_bytes=B] + │ SortExec: expr=[s_suppkey@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=1, elapsed_compute=, output_bytes=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1, elapsed_compute=, output_bytes=B] + │ 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], metrics=[output_rows=1, elapsed_compute=, output_bytes=B, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ AggregateExec: mode=Final, gby=[], aggr=[max(revenue0.total_revenue)], metrics=[output_rows=1, elapsed_compute=, output_bytes=] + │ CoalescePartitionsExec, metrics=[output_rows=12, elapsed_compute=, output_bytes=] │ [Stage 2] => NetworkCoalesceExec: output_partitions=12, input_tasks=2, metrics=[] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1000, elapsed_compute=, output_bytes=3.5 MB] - │ 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], metrics=[output_rows=1000, elapsed_compute=, output_bytes=481.7 KB, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=, output_bytes=127.4 KB] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/supplier/1.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/10.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/12.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/13.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/15.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/16.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/3.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/4.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/6.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/7.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_name, s_address, s_phone], file_type=parquet, metrics=[output_rows=1000, elapsed_compute=, output_bytes=127.4 KB, files_ranges_pruned_statistics= total → X matched, row_groups_pruned_statistics= total → X matched, row_groups_pruned_bloom_filter= total → X matched, page_index_rows_pruned= total → X matched, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] - │ ProjectionExec: expr=[l_suppkey@0 as supplier_no, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue], metrics=[output_rows=1000, elapsed_compute=, output_bytes=27.6 KB] - │ AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=1000, elapsed_compute=, output_bytes=27.6 KB, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5825, elapsed_compute=, output_bytes=2.2 MB] - │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1000, elapsed_compute=, output_bytes=B] + │ 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], metrics=[output_rows=1000, elapsed_compute=, output_bytes=B, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=, output_bytes=B] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/supplier/1.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/10.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/12.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/13.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/15.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/16.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/3.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/4.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/6.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/7.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_name, s_address, s_phone], file_type=parquet, metrics=[output_rows=1000, elapsed_compute=, output_bytes=B, files_ranges_pruned_statistics= total → X matched, row_groups_pruned_statistics= total → X matched, row_groups_pruned_bloom_filter= total → X matched, page_index_rows_pruned= total → X matched, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ ProjectionExec: expr=[l_suppkey@0 as supplier_no, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue], metrics=[output_rows=1000, elapsed_compute=, output_bytes=B] + │ AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=1000, elapsed_compute=, output_bytes=B, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[max(revenue0.total_revenue)], metrics=[output_rows=12, elapsed_compute=] │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue], metrics=[output_rows=1000, elapsed_compute=] │ AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=1000, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5825, elapsed_compute=] - │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] + │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] - │ RepartitionExec: partitioning=Hash([l_suppkey@0], 12), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=5825, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=22830, elapsed_compute=] - │ FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2], metrics=[output_rows=22830, elapsed_compute=] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1996-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1996-04-01, required_guarantees=[], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5825, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([l_suppkey@0], 12), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=5825, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=22830, elapsed_compute=] + │ FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2], metrics=[output_rows=22830, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1996-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1996-04-01, required_guarantees=[], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ RepartitionExec: partitioning=Hash([l_suppkey@0], 6), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=5825, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=22830, elapsed_compute=] - │ FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2], metrics=[output_rows=22830, elapsed_compute=] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1996-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1996-04-01, required_guarantees=[], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5825, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([l_suppkey@0], 6), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=5825, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=22830, elapsed_compute=] + │ FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2], metrics=[output_rows=22830, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1996-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1996-04-01, required_guarantees=[], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── "); Ok(()) @@ -809,36 +809,36 @@ mod tests { let plan = test_tpch_query(16).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] - │ SortPreservingMergeExec: [supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST], metrics=[output_rows=2762, elapsed_compute=, output_bytes=297.3 KB] + │ SortPreservingMergeExec: [supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST], metrics=[output_rows=2762, elapsed_compute=, output_bytes=B] │ [Stage 5] => NetworkCoalesceExec: output_partitions=12, input_tasks=2, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] │ SortExec: expr=[supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=2762, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ 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], metrics=[output_rows=2762, elapsed_compute=] │ 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)], metrics=[output_rows=2762, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=10538, elapsed_compute=] - │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] - │ RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2], 12), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ 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)], metrics=[output_rows=10538, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ 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=[], metrics=[output_rows=11632, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=11634, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=10538, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2], 12), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ 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)], metrics=[output_rows=10538, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] + │ 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=[], metrics=[output_rows=11632, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p17] - │ RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ 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=[], metrics=[output_rows=11634, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=11635, elapsed_compute=] - │ HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(s_suppkey@0, ps_suppkey@0)], metrics=[output_rows=11635, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=1, elapsed_compute=] - │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] - │ 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], metrics=[output_rows=11644, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=11644, elapsed_compute=] - │ 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], metrics=[output_rows=11644, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=2911, elapsed_compute=] - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/partsupp/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=80000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=11634, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ 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=[], metrics=[output_rows=11634, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=11635, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(s_suppkey@0, ps_suppkey@0)], metrics=[output_rows=11635, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=1, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ 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], metrics=[output_rows=11644, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=11644, elapsed_compute=] + │ 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], metrics=[output_rows=11644, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=2911, elapsed_compute=] + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/partsupp/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=80000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1, elapsed_compute=] @@ -861,30 +861,29 @@ mod tests { let plan = test_tpch_query(17).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] - │ ProjectionExec: expr=[CAST(sum(lineitem.l_extendedprice)@0 AS Float64) / 7 as avg_yearly], metrics=[output_rows=1, elapsed_compute=, output_bytes=8.0 B] - │ AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice)], metrics=[output_rows=1, elapsed_compute=, output_bytes=16.0 B] - │ CoalescePartitionsExec, metrics=[output_rows=24, elapsed_compute=, output_bytes=3.0 KB] + │ ProjectionExec: expr=[CAST(sum(lineitem.l_extendedprice)@0 AS Float64) / 7 as avg_yearly], metrics=[output_rows=1, elapsed_compute=, output_bytes=] + │ AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice)], metrics=[output_rows=1, elapsed_compute=, output_bytes=] + │ CoalescePartitionsExec, metrics=[output_rows=24, elapsed_compute=, output_bytes=B] │ [Stage 4] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice)], metrics=[output_rows=24, elapsed_compute=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=43, elapsed_compute=] │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * avg(lineitem.l_quantity)@1, projection=[l_extendedprice@1], metrics=[output_rows=43, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=555, elapsed_compute=] - │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] │ 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], metrics=[output_rows=20000, elapsed_compute=] │ AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)], metrics=[output_rows=20000, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=118823, elapsed_compute=] - │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p23] - │ RepartitionExec: partitioning=Hash([p_partkey@2], 24), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, p_partkey@0 as p_partkey], metrics=[output_rows=555, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=555, elapsed_compute=] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], projection=[p_partkey@0, l_quantity@2, l_extendedprice@3], metrics=[output_rows=555, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=18, elapsed_compute=] - │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_quantity, l_extendedprice], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=555, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([p_partkey@2], 24), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, p_partkey@0 as p_partkey], metrics=[output_rows=555, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=555, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], projection=[p_partkey@0, l_quantity@2, l_extendedprice@3], metrics=[output_rows=555, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=18, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_quantity, l_extendedprice], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=18, elapsed_compute=] @@ -893,10 +892,11 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/part/1.parquet, /testdata/tpch/explain_analyze_sf0.1/part/10.parquet, /testdata/tpch/explain_analyze_sf0.1/part/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/12.parquet, /testdata/tpch/explain_analyze_sf0.1/part/13.parquet, /testdata/tpch/explain_analyze_sf0.1/part/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/15.parquet, /testdata/tpch/explain_analyze_sf0.1/part/16.parquet, /testdata/tpch/explain_analyze_sf0.1/part/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/3.parquet, /testdata/tpch/explain_analyze_sf0.1/part/4.parquet, /testdata/tpch/explain_analyze_sf0.1/part/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/6.parquet, /testdata/tpch/explain_analyze_sf0.1/part/7.parquet, /testdata/tpch/explain_analyze_sf0.1/part/8.parquet], ...]}, projection=[p_partkey, p_brand, p_container], file_type=parquet, predicate=p_brand@1 = Brand#23 AND p_container@2 = MED BOX, pruning_predicate=p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#23 AND Brand#23 <= p_brand_max@1 AND p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED BOX AND MED BOX <= p_container_max@5, required_guarantees=[p_brand in (Brand#23), p_container in (MED BOX)], metrics=[output_rows=20000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] - │ RepartitionExec: partitioning=Hash([l_partkey@0], 24), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)], metrics=[output_rows=118823, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_quantity], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=118823, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([l_partkey@0], 24), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)], metrics=[output_rows=118823, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_quantity], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── "); Ok(()) @@ -907,65 +907,65 @@ mod tests { let plan = test_tpch_query(18).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] - │ SortPreservingMergeExec: [o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST], metrics=[output_rows=5, elapsed_compute=, output_bytes=4.4 KB] + │ SortPreservingMergeExec: [o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST], metrics=[output_rows=5, elapsed_compute=, output_bytes=B] │ [Stage 8] => NetworkCoalesceExec: output_partitions=18, input_tasks=3, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ SortExec: expr=[o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=5, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ 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)], metrics=[output_rows=5, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5, elapsed_compute=] - │ [Stage 7] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + │ [Stage 7] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 7 ── Tasks: t0:[p0..p17] - │ RepartitionExec: partitioning=Hash([c_name@0, c_custkey@1, o_orderkey@2, o_orderdate@3, o_totalprice@4], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ 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)], metrics=[output_rows=5, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=35, elapsed_compute=] - │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(l_orderkey@0, o_orderkey@2)], metrics=[output_rows=35, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=5, elapsed_compute=] - │ [Stage 2] => NetworkCoalesceExec: output_partitions=18, input_tasks=3, metrics=[] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=600572, elapsed_compute=] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@2, l_orderkey@0)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@3, o_orderdate@4, l_quantity@6], metrics=[output_rows=600572, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=150000, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([c_name@0, c_custkey@1, o_orderkey@2, o_orderdate@3, o_totalprice@4], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ 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)], metrics=[output_rows=5, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=35, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(l_orderkey@0, o_orderkey@2)], metrics=[output_rows=35, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=5, elapsed_compute=] + │ [Stage 2] => NetworkCoalesceExec: output_partitions=18, input_tasks=3, metrics=[] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=600572, elapsed_compute=] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@2, l_orderkey@0)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@3, o_orderdate@4, l_quantity@6], metrics=[output_rows=600572, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=600572, elapsed_compute=] │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5, elapsed_compute=] │ FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0], metrics=[output_rows=5, elapsed_compute=] │ AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)], metrics=[output_rows=150000, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=150000, elapsed_compute=] - │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] + │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] - │ RepartitionExec: partitioning=Hash([l_orderkey@0], 18), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)], metrics=[output_rows=150000, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_quantity], file_type=parquet, metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=150000, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 18), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)], metrics=[output_rows=150000, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_quantity], file_type=parquet, metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ RepartitionExec: partitioning=Hash([o_orderkey@2], 6), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=150000, elapsed_compute=] - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@4, o_orderdate@5], metrics=[output_rows=150000, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=15000, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=150000, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([o_orderkey@2], 6), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=150000, elapsed_compute=] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@4, o_orderdate@5], metrics=[output_rows=150000, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=150000, elapsed_compute=] │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] - │ RepartitionExec: partitioning=Hash([c_custkey@0], 24), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/customer/1.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/10.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/12.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/13.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/15.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/16.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/3.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/4.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/6.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/7.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/8.parquet], ...]}, projection=[c_custkey, c_name], file_type=parquet, metrics=[output_rows=15000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=15000, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([c_custkey@0], 24), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/customer/1.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/10.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/12.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/13.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/15.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/16.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/3.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/4.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/customer/6.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/7.parquet, /testdata/tpch/explain_analyze_sf0.1/customer/8.parquet], ...]}, projection=[c_custkey, c_name], file_type=parquet, metrics=[output_rows=15000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] - │ RepartitionExec: partitioning=Hash([o_custkey@1], 24), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/orders/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=150000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=150000, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([o_custkey@1], 24), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/orders/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=150000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ RepartitionExec: partitioning=Hash([l_orderkey@0], 6), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_quantity], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=600572, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 6), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_quantity], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── "); Ok(()) @@ -976,17 +976,17 @@ mod tests { let plan = test_tpch_query(19).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] - │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@0 as revenue], metrics=[output_rows=1, elapsed_compute=, output_bytes=16.0 B] - │ AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=1, elapsed_compute=, output_bytes=16.0 B] - │ CoalescePartitionsExec, metrics=[output_rows=6, elapsed_compute=, output_bytes=160.0 B] - │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=6, elapsed_compute=, output_bytes=160.0 B] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=10, elapsed_compute=, output_bytes=1280.0 KB] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], filter=p_brand@1 = Brand#12 AND p_container@3 IN ([SM CASE, SM BOX, SM PACK, SM PKG]) AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN ([MED BAG, MED BOX, MED PKG, MED PACK]) AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN ([LG CASE, LG BOX, LG PACK, LG PKG]) AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_extendedprice@6, l_discount@7], metrics=[output_rows=10, elapsed_compute=, output_bytes=320.0 B, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=42, elapsed_compute=, output_bytes=4.0 KB] + │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@0 as revenue], metrics=[output_rows=1, elapsed_compute=, output_bytes=] + │ AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=1, elapsed_compute=, output_bytes=] + │ CoalescePartitionsExec, metrics=[output_rows=6, elapsed_compute=, output_bytes=] + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=6, elapsed_compute=, output_bytes=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=10, elapsed_compute=, output_bytes=B] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], filter=p_brand@1 = Brand#12 AND p_container@3 IN ([SM CASE, SM BOX, SM PACK, SM PKG]) AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN ([MED BAG, MED BOX, MED PKG, MED PACK]) AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN ([LG CASE, LG BOX, LG PACK, LG PKG]) AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_extendedprice@6, l_discount@7], metrics=[output_rows=10, elapsed_compute=, output_bytes=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=42, elapsed_compute=, output_bytes=B] │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=12635, elapsed_compute=, output_bytes=5.2 MB] - │ 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], metrics=[output_rows=12635, elapsed_compute=, output_bytes=691.0 KB, selectivity=2.1% (12635/600572)] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], file_type=parquet, predicate=(l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2) AND (l_shipmode@5 = AIR OR l_shipmode@5 = AIR REG) AND l_shipinstruct@4 = DELIVER IN PERSON AND DynamicFilter [ l_partkey@0 >= 55 AND l_partkey@0 <= 19916 ], pruning_predicate=(l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(100),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(1100),15,2 OR l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(1000),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(2000),15,2 OR l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(2000),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(3000),15,2) AND (l_shipmode_null_count@6 != row_count@2 AND l_shipmode_min@4 <= AIR AND AIR <= l_shipmode_max@5 OR l_shipmode_null_count@6 != row_count@2 AND l_shipmode_min@4 <= AIR REG AND AIR REG <= l_shipmode_max@5) AND l_shipinstruct_null_count@9 != row_count@2 AND l_shipinstruct_min@7 <= DELIVER IN PERSON AND DELIVER IN PERSON <= l_shipinstruct_max@8 AND l_partkey_null_count@11 != row_count@2 AND l_partkey_max@10 >= 55 AND l_partkey_null_count@11 != row_count@2 AND l_partkey_min@12 <= 19916, required_guarantees=[l_shipinstruct in (DELIVER IN PERSON), l_shipmode in (AIR, AIR REG)], metrics=[output_rows=600572, elapsed_compute=, output_bytes=52.1 MB, files_ranges_pruned_statistics= total → X matched, row_groups_pruned_statistics= total → X matched, row_groups_pruned_bloom_filter= total → X matched, page_index_rows_pruned= total → X matched, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=12635, elapsed_compute=, output_bytes=B] + │ 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], metrics=[output_rows=12635, elapsed_compute=, output_bytes=B, selectivity=2.1% (12635/600572)] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], file_type=parquet, predicate=(l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2) AND (l_shipmode@5 = AIR OR l_shipmode@5 = AIR REG) AND l_shipinstruct@4 = DELIVER IN PERSON AND DynamicFilter [ l_partkey@0 >= 55 AND l_partkey@0 <= 19916 ], pruning_predicate=(l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(100),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(1100),15,2 OR l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(1000),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(2000),15,2 OR l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(2000),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(3000),15,2) AND (l_shipmode_null_count@6 != row_count@2 AND l_shipmode_min@4 <= AIR AND AIR <= l_shipmode_max@5 OR l_shipmode_null_count@6 != row_count@2 AND l_shipmode_min@4 <= AIR REG AND AIR REG <= l_shipmode_max@5) AND l_shipinstruct_null_count@9 != row_count@2 AND l_shipinstruct_min@7 <= DELIVER IN PERSON AND DELIVER IN PERSON <= l_shipinstruct_max@8 AND l_partkey_null_count@11 != row_count@2 AND l_partkey_max@10 >= 55 AND l_partkey_null_count@11 != row_count@2 AND l_partkey_min@12 <= 19916, required_guarantees=[l_shipinstruct in (DELIVER IN PERSON), l_shipmode in (AIR, AIR REG)], metrics=[output_rows=600572, elapsed_compute=, output_bytes=B, files_ranges_pruned_statistics= total → X matched, row_groups_pruned_statistics= total → X matched, row_groups_pruned_bloom_filter= total → X matched, page_index_rows_pruned= total → X matched, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=42, elapsed_compute=] @@ -1003,28 +1003,27 @@ mod tests { let plan = test_tpch_query(20).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] - │ SortPreservingMergeExec: [s_name@0 ASC NULLS LAST], metrics=[output_rows=144, elapsed_compute=, output_bytes=20.5 KB] - │ SortExec: expr=[s_name@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=144, elapsed_compute=, output_bytes=0.0 B, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=144, elapsed_compute=, output_bytes=272.0 KB] - │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_name@1, s_address@2], metrics=[output_rows=144, elapsed_compute=, output_bytes=154.5 KB, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=592, elapsed_compute=, output_bytes=3.6 MB] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=592, elapsed_compute=, output_bytes=3.6 MB] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[s_suppkey@1, s_name@2, s_address@3], metrics=[output_rows=592, elapsed_compute=, output_bytes=70.0 KB, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=16, elapsed_compute=, output_bytes=768.0 B] + │ SortPreservingMergeExec: [s_name@0 ASC NULLS LAST], metrics=[output_rows=144, elapsed_compute=, output_bytes=B] + │ SortExec: expr=[s_name@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=144, elapsed_compute=, output_bytes=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=144, elapsed_compute=, output_bytes=B] + │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_name@1, s_address@2], metrics=[output_rows=144, elapsed_compute=, output_bytes=B, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=592, elapsed_compute=, output_bytes=B] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=592, elapsed_compute=, output_bytes=B] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[s_suppkey@1, s_name@2, s_address@3], metrics=[output_rows=592, elapsed_compute=, output_bytes=B, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=16, elapsed_compute=, output_bytes=] │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/supplier/1.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/10.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/12.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/13.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/15.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/16.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/3.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/4.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/6.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/7.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_name, s_address, s_nationkey], file_type=parquet, predicate=DynamicFilter [ s_nationkey@3 >= 3 AND s_nationkey@3 <= 3 ], pruning_predicate=s_nationkey_null_count@1 != row_count@2 AND s_nationkey_max@0 >= 3 AND s_nationkey_null_count@1 != row_count@2 AND s_nationkey_min@3 <= 3, required_guarantees=[], metrics=[output_rows=1000, elapsed_compute=, output_bytes=99.6 KB, files_ranges_pruned_statistics= total → X matched, row_groups_pruned_statistics= total → X matched, row_groups_pruned_bloom_filter= total → X matched, page_index_rows_pruned= total → X matched, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=521, elapsed_compute=, output_bytes=768.0 KB] - │ 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], metrics=[output_rows=521, elapsed_compute=, output_bytes=4.1 KB, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=760, elapsed_compute=, output_bytes=1920.0 KB] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=760, elapsed_compute=, output_bytes=1920.0 KB] - │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(p_partkey@0, ps_partkey@0)], metrics=[output_rows=760, elapsed_compute=, output_bytes=14.8 KB, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=190, elapsed_compute=, output_bytes=2.1 KB] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/supplier/1.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/10.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/12.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/13.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/15.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/16.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/3.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/4.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/6.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/7.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_name, s_address, s_nationkey], file_type=parquet, predicate=DynamicFilter [ s_nationkey@3 >= 3 AND s_nationkey@3 <= 3 ], pruning_predicate=s_nationkey_null_count@1 != row_count@2 AND s_nationkey_max@0 >= 3 AND s_nationkey_null_count@1 != row_count@2 AND s_nationkey_min@3 <= 3, required_guarantees=[], metrics=[output_rows=1000, elapsed_compute=, output_bytes=B, files_ranges_pruned_statistics= total → X matched, row_groups_pruned_statistics= total → X matched, row_groups_pruned_bloom_filter= total → X matched, page_index_rows_pruned= total → X matched, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=521, elapsed_compute=, output_bytes=B] + │ 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], metrics=[output_rows=521, elapsed_compute=, output_bytes=B, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=760, elapsed_compute=, output_bytes=B] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=760, elapsed_compute=, output_bytes=B] + │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(p_partkey@0, ps_partkey@0)], metrics=[output_rows=760, elapsed_compute=, output_bytes=B, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=190, elapsed_compute=, output_bytes=B] │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/partsupp/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_availqty], file_type=parquet, metrics=[output_rows=80000, elapsed_compute=, output_bytes=1562.5 KB, files_ranges_pruned_statistics= total → X matched, row_groups_pruned_statistics= total → X matched, row_groups_pruned_bloom_filter= total → X matched, page_index_rows_pruned= total → X matched, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] - │ 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], metrics=[output_rows=54539, elapsed_compute=, output_bytes=3.4 MB] - │ AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)], metrics=[output_rows=54539, elapsed_compute=, output_bytes=5.4 MB, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=83515, elapsed_compute=, output_bytes=6.0 MB] - │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/partsupp/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/partsupp/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_availqty], file_type=parquet, metrics=[output_rows=80000, elapsed_compute=, output_bytes=B, files_ranges_pruned_statistics= total → X matched, row_groups_pruned_statistics= total → X matched, row_groups_pruned_bloom_filter= total → X matched, page_index_rows_pruned= total → X matched, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ 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], metrics=[output_rows=54539, elapsed_compute=, output_bytes=B] + │ AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)], metrics=[output_rows=54539, elapsed_compute=, output_bytes=B, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=4, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16, elapsed_compute=] @@ -1039,12 +1038,13 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/part/1.parquet, /testdata/tpch/explain_analyze_sf0.1/part/10.parquet, /testdata/tpch/explain_analyze_sf0.1/part/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/12.parquet, /testdata/tpch/explain_analyze_sf0.1/part/13.parquet, /testdata/tpch/explain_analyze_sf0.1/part/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/15.parquet, /testdata/tpch/explain_analyze_sf0.1/part/16.parquet, /testdata/tpch/explain_analyze_sf0.1/part/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/3.parquet, /testdata/tpch/explain_analyze_sf0.1/part/4.parquet, /testdata/tpch/explain_analyze_sf0.1/part/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/part/6.parquet, /testdata/tpch/explain_analyze_sf0.1/part/7.parquet, /testdata/tpch/explain_analyze_sf0.1/part/8.parquet], ...]}, projection=[p_partkey, p_name], file_type=parquet, predicate=p_name@1 LIKE forest%, pruning_predicate=p_name_null_count@2 != row_count@3 AND p_name_min@0 <= foresu AND forest <= p_name_max@1, required_guarantees=[], metrics=[output_rows=20000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 6), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)], metrics=[output_rows=83515, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=92040, elapsed_compute=] - │ FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2], metrics=[output_rows=92040, elapsed_compute=] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND DynamicFilter [ l_partkey@0 >= 5 AND l_partkey@0 <= 19852 AND l_suppkey@1 >= 4 AND l_suppkey@1 <= 1000 ], 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_partkey_null_count@5 != row_count@2 AND l_partkey_max@4 >= 5 AND l_partkey_null_count@5 != row_count@2 AND l_partkey_min@6 <= 19852 AND l_suppkey_null_count@8 != row_count@2 AND l_suppkey_max@7 >= 4 AND l_suppkey_null_count@8 != row_count@2 AND l_suppkey_min@9 <= 1000, required_guarantees=[], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=83515, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 6), input_partitions=2, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)], metrics=[output_rows=83515, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=92040, elapsed_compute=] + │ FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2], metrics=[output_rows=92040, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND DynamicFilter [ l_partkey@0 >= 5 AND l_partkey@0 <= 19852 AND l_suppkey@1 >= 4 AND l_suppkey@1 <= 1000 ], 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_partkey_null_count@5 != row_count@2 AND l_partkey_max@4 >= 5 AND l_partkey_null_count@5 != row_count@2 AND l_partkey_min@6 <= 19852 AND l_suppkey_null_count@8 != row_count@2 AND l_suppkey_max@7 >= 4 AND l_suppkey_null_count@8 != row_count@2 AND l_suppkey_min@9 <= 1000, required_guarantees=[], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── "); Ok(()) @@ -1055,44 +1055,44 @@ mod tests { let plan = test_tpch_query(21).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] - │ SortPreservingMergeExec: [numwait@1 DESC, s_name@0 ASC NULLS LAST], metrics=[output_rows=47, elapsed_compute=, output_bytes=6.9 KB] + │ SortPreservingMergeExec: [numwait@1 DESC, s_name@0 ASC NULLS LAST], metrics=[output_rows=47, elapsed_compute=, output_bytes=B] │ [Stage 4] => NetworkCoalesceExec: output_partitions=18, input_tasks=3, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ SortExec: expr=[numwait@1 DESC, s_name@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=47, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ ProjectionExec: expr=[s_name@0 as s_name, count(Int64(1))@1 as numwait], metrics=[output_rows=47, elapsed_compute=] │ AggregateExec: mode=FinalPartitioned, gby=[s_name@0 as s_name], aggr=[count(Int64(1))], metrics=[output_rows=47, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=47, elapsed_compute=] - │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p17] - │ RepartitionExec: partitioning=Hash([s_name@0], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[count(Int64(1))], metrics=[output_rows=47, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=7440, elapsed_compute=] - │ HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, projection=[s_name@0], metrics=[output_rows=7440, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=132720, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=132720, elapsed_compute=] - │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, metrics=[output_rows=132720, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=137440, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=137440, elapsed_compute=] - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@1)], projection=[s_name@1, l_orderkey@3, l_suppkey@4], metrics=[output_rows=137440, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=16, elapsed_compute=] - │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=182902, elapsed_compute=] - │ 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], metrics=[output_rows=182902, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=72884, elapsed_compute=] - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=379809, elapsed_compute=] - │ 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], metrics=[output_rows=379809, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/supplier/1.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/10.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/12.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/13.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/15.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/16.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/3.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/4.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/6.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/7.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_name, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=1000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=379809, elapsed_compute=] - │ FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1], metrics=[output_rows=379809, elapsed_compute=] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@3 > l_commitdate@2 AND DynamicFilter [ empty ] AND DynamicFilter [ empty ], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_suppkey], file_type=parquet, metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=379809, elapsed_compute=] - │ FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1], metrics=[output_rows=379809, elapsed_compute=] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@3 > l_commitdate@2, metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=47, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([s_name@0], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[count(Int64(1))], metrics=[output_rows=47, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=7440, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, projection=[s_name@0], metrics=[output_rows=7440, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=132720, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=132720, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, metrics=[output_rows=132720, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=137440, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=137440, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@1)], projection=[s_name@1, l_orderkey@3, l_suppkey@4], metrics=[output_rows=137440, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=16, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=182902, elapsed_compute=] + │ 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], metrics=[output_rows=182902, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=72884, elapsed_compute=] + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=379809, elapsed_compute=] + │ 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], metrics=[output_rows=379809, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/supplier/1.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/10.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/11.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/12.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/13.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/14.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/15.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/16.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/2.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/3.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/4.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/5.parquet], [/testdata/tpch/explain_analyze_sf0.1/supplier/6.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/7.parquet, /testdata/tpch/explain_analyze_sf0.1/supplier/8.parquet], ...]}, projection=[s_suppkey, s_name, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ], metrics=[output_rows=1000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=379809, elapsed_compute=] + │ FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1], metrics=[output_rows=379809, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@3 > l_commitdate@2 AND DynamicFilter [ empty ] AND DynamicFilter [ empty ], metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_suppkey], file_type=parquet, metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=379809, elapsed_compute=] + │ FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1], metrics=[output_rows=379809, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/lineitem/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/lineitem/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@3 > l_commitdate@2, metrics=[output_rows=600572, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16, elapsed_compute=] @@ -1115,30 +1115,30 @@ mod tests { let plan = test_tpch_query(22).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] - │ SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST], metrics=[output_rows=7, elapsed_compute=, output_bytes=280.0 B] + │ SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST], metrics=[output_rows=7, elapsed_compute=, output_bytes=] │ [Stage 4] => NetworkCoalesceExec: output_partitions=18, input_tasks=3, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ SortExec: expr=[cntrycode@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=7, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ ProjectionExec: expr=[cntrycode@0 as cntrycode, count(Int64(1))@1 as numcust, sum(custsale.c_acctbal)@2 as totacctbal], metrics=[output_rows=7, elapsed_compute=] │ AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)], metrics=[output_rows=7, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=7, elapsed_compute=] - │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p17] - │ RepartitionExec: partitioning=Hash([cntrycode@0], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] - │ AggregateExec: mode=Partial, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)], metrics=[output_rows=7, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] - │ ProjectionExec: expr=[substr(c_phone@1, 1, 2) as cntrycode, c_acctbal@2 as c_acctbal], metrics=[output_rows=641, elapsed_compute=] - │ NestedLoopJoinExec: join_type=Inner, filter=join_proj_push_down_1@1 > avg(customer.c_acctbal)@0, projection=[avg(customer.c_acctbal)@0, c_phone@1, c_acctbal@2], metrics=[output_rows=641, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)], metrics=[output_rows=1, elapsed_compute=] - │ CoalescePartitionsExec, metrics=[output_rows=8, elapsed_compute=] - │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] - │ ProjectionExec: expr=[c_phone@0 as c_phone, c_acctbal@1 as c_acctbal, CAST(c_acctbal@1 AS Decimal128(19, 6)) as join_proj_push_down_1], metrics=[output_rows=1360, elapsed_compute=] - │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1360, elapsed_compute=] - │ HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)], projection=[c_phone@1, c_acctbal@2], metrics=[output_rows=1360, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] - │ CoalescePartitionsExec, metrics=[output_rows=4115, elapsed_compute=] - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/orders/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/7.parquet:..], ...]}, projection=[o_custkey], file_type=parquet, metrics=[output_rows=150000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=7, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([cntrycode@0], 18), input_partitions=6, metrics=[spill_count=, spilled_bytes=, spilled_rows=, fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)], metrics=[output_rows=7, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=, aggregate_arguments_time=, aggregation_time=, emitting_time=, time_calculating_group_ids=] + │ ProjectionExec: expr=[substr(c_phone@1, 1, 2) as cntrycode, c_acctbal@2 as c_acctbal], metrics=[output_rows=641, elapsed_compute=] + │ NestedLoopJoinExec: join_type=Inner, filter=join_proj_push_down_1@1 > avg(customer.c_acctbal)@0, projection=[avg(customer.c_acctbal)@0, c_phone@1, c_acctbal@2], metrics=[output_rows=641, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)], metrics=[output_rows=1, elapsed_compute=] + │ CoalescePartitionsExec, metrics=[output_rows=8, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ ProjectionExec: expr=[c_phone@0 as c_phone, c_acctbal@1 as c_acctbal, CAST(c_acctbal@1 AS Decimal128(19, 6)) as join_proj_push_down_1], metrics=[output_rows=1360, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1360, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)], projection=[c_phone@1, c_acctbal@2], metrics=[output_rows=1360, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=4115, elapsed_compute=] + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/explain_analyze_sf0.1/orders/1.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/10.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/11.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/12.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/13.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/14.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/15.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/16.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/2.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/3.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:..], [/testdata/tpch/explain_analyze_sf0.1/orders/4.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/5.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/6.parquet:.., /testdata/tpch/explain_analyze_sf0.1/orders/7.parquet:..], ...]}, projection=[o_custkey], file_type=parquet, metrics=[output_rows=150000, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, num_predicate_creation_errors=, predicate_cache_inner_records=0, predicate_cache_records=0, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, bloom_filter_eval_time=, metadata_load_time=, page_index_eval_time=, row_pushdown_eval_time=, statistics_eval_time=, time_elapsed_opening=, time_elapsed_processing=, time_elapsed_scanning_total=, time_elapsed_scanning_until_data=] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)], metrics=[output_rows=8, elapsed_compute=] diff --git a/tests/tpch_plans_test.rs b/tests/tpch_plans_test.rs index e440fdf6..667274f1 100644 --- a/tests/tpch_plans_test.rs +++ b/tests/tpch_plans_test.rs @@ -25,17 +25,17 @@ mod tests { │ 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 - │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] - │ RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 12), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[l_returnflag@5 as l_returnflag, l_linestatus@6 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(Int64(1))] - │ ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as __common_expr_1, l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: l_shipdate@6 <= 1998-09-02, projection=[l_quantity@0, l_extendedprice@1, l_discount@2, l_tax@3, l_returnflag@4, l_linestatus@5] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_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=[] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 12), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[l_returnflag@5 as l_returnflag, l_linestatus@6 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(Int64(1))] + │ ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as __common_expr_1, l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: l_shipdate@6 <= 1998-09-02, projection=[l_quantity@0, l_extendedprice@1, l_discount@2, l_tax@3, l_returnflag@4, l_linestatus@5] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_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(()) @@ -54,32 +54,31 @@ mod tests { │ ProjectionExec: expr=[s_acctbal@5 as s_acctbal, s_name@2 as s_name, n_name@7 as n_name, p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_address@3 as s_address, s_phone@4 as s_phone, s_comment@6 as s_comment] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@1), (ps_supplycost@7, min(partsupp.ps_supplycost)@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, n_name@8] - │ CoalesceBatchesExec: target_batch_size=8192 - │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1 - │ CoalesceBatchesExec: target_batch_size=8192 - │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1 + │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p23] - │ RepartitionExec: partitioning=Hash([p_partkey@0, ps_supplycost@7], 24), input_partitions=6 - │ CoalesceBatchesExec: target_batch_size=8192 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@9)], projection=[p_partkey@1, p_mfgr@2, s_name@3, s_address@4, s_phone@5, s_acctbal@6, s_comment@7, ps_supplycost@8, n_name@9] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 - │ ProjectionExec: expr=[p_partkey@2 as p_partkey, p_mfgr@3 as p_mfgr, s_name@4 as s_name, s_address@5 as s_address, s_phone@6 as s_phone, s_acctbal@7 as s_acctbal, s_comment@8 as s_comment, ps_supplycost@9 as ps_supplycost, n_name@0 as n_name, n_regionkey@1 as n_regionkey] - │ CoalesceBatchesExec: target_batch_size=8192 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@4)], projection=[n_name@1, n_regionkey@2, p_partkey@3, p_mfgr@4, s_name@5, s_address@6, s_phone@8, s_acctbal@9, s_comment@10, ps_supplycost@11] - │ CoalescePartitionsExec - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet, /testdata/tpch/plan_sf0.02/nation/10.parquet, /testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet, /testdata/tpch/plan_sf0.02/nation/13.parquet, /testdata/tpch/plan_sf0.02/nation/14.parquet], [/testdata/tpch/plan_sf0.02/nation/15.parquet, /testdata/tpch/plan_sf0.02/nation/16.parquet, /testdata/tpch/plan_sf0.02/nation/2.parquet], [/testdata/tpch/plan_sf0.02/nation/3.parquet, /testdata/tpch/plan_sf0.02/nation/4.parquet, /testdata/tpch/plan_sf0.02/nation/5.parquet], [/testdata/tpch/plan_sf0.02/nation/6.parquet, /testdata/tpch/plan_sf0.02/nation/7.parquet, /testdata/tpch/plan_sf0.02/nation/8.parquet], ...]}, projection=[n_nationkey, n_name, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ] - │ 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={6 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet, /testdata/tpch/plan_sf0.02/supplier/10.parquet, /testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet, /testdata/tpch/plan_sf0.02/supplier/13.parquet, /testdata/tpch/plan_sf0.02/supplier/14.parquet], [/testdata/tpch/plan_sf0.02/supplier/15.parquet, /testdata/tpch/plan_sf0.02/supplier/16.parquet, /testdata/tpch/plan_sf0.02/supplier/2.parquet], [/testdata/tpch/plan_sf0.02/supplier/3.parquet, /testdata/tpch/plan_sf0.02/supplier/4.parquet, /testdata/tpch/plan_sf0.02/supplier/5.parquet], [/testdata/tpch/plan_sf0.02/supplier/6.parquet, /testdata/tpch/plan_sf0.02/supplier/7.parquet, /testdata/tpch/plan_sf0.02/supplier/8.parquet], ...]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], file_type=parquet, predicate=DynamicFilter [ empty ] - │ 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 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/partsupp/1.parquet, /testdata/tpch/plan_sf0.02/partsupp/10.parquet, /testdata/tpch/plan_sf0.02/partsupp/11.parquet], [/testdata/tpch/plan_sf0.02/partsupp/12.parquet, /testdata/tpch/plan_sf0.02/partsupp/13.parquet, /testdata/tpch/plan_sf0.02/partsupp/14.parquet], [/testdata/tpch/plan_sf0.02/partsupp/15.parquet, /testdata/tpch/plan_sf0.02/partsupp/16.parquet, /testdata/tpch/plan_sf0.02/partsupp/2.parquet], [/testdata/tpch/plan_sf0.02/partsupp/3.parquet, /testdata/tpch/plan_sf0.02/partsupp/4.parquet, /testdata/tpch/plan_sf0.02/partsupp/5.parquet], [/testdata/tpch/plan_sf0.02/partsupp/6.parquet, /testdata/tpch/plan_sf0.02/partsupp/7.parquet, /testdata/tpch/plan_sf0.02/partsupp/8.parquet], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([p_partkey@0, ps_supplycost@7], 24), input_partitions=6 + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@9)], projection=[p_partkey@1, p_mfgr@2, s_name@3, s_address@4, s_phone@5, s_acctbal@6, s_comment@7, ps_supplycost@8, n_name@9] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[p_partkey@2 as p_partkey, p_mfgr@3 as p_mfgr, s_name@4 as s_name, s_address@5 as s_address, s_phone@6 as s_phone, s_acctbal@7 as s_acctbal, s_comment@8 as s_comment, ps_supplycost@9 as ps_supplycost, n_name@0 as n_name, n_regionkey@1 as n_regionkey] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@4)], projection=[n_name@1, n_regionkey@2, p_partkey@3, p_mfgr@4, s_name@5, s_address@6, s_phone@8, s_acctbal@9, s_comment@10, ps_supplycost@11] + │ CoalescePartitionsExec + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet, /testdata/tpch/plan_sf0.02/nation/10.parquet, /testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet, /testdata/tpch/plan_sf0.02/nation/13.parquet, /testdata/tpch/plan_sf0.02/nation/14.parquet], [/testdata/tpch/plan_sf0.02/nation/15.parquet, /testdata/tpch/plan_sf0.02/nation/16.parquet, /testdata/tpch/plan_sf0.02/nation/2.parquet], [/testdata/tpch/plan_sf0.02/nation/3.parquet, /testdata/tpch/plan_sf0.02/nation/4.parquet, /testdata/tpch/plan_sf0.02/nation/5.parquet], [/testdata/tpch/plan_sf0.02/nation/6.parquet, /testdata/tpch/plan_sf0.02/nation/7.parquet, /testdata/tpch/plan_sf0.02/nation/8.parquet], ...]}, projection=[n_nationkey, n_name, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ] + │ 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={6 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet, /testdata/tpch/plan_sf0.02/supplier/10.parquet, /testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet, /testdata/tpch/plan_sf0.02/supplier/13.parquet, /testdata/tpch/plan_sf0.02/supplier/14.parquet], [/testdata/tpch/plan_sf0.02/supplier/15.parquet, /testdata/tpch/plan_sf0.02/supplier/16.parquet, /testdata/tpch/plan_sf0.02/supplier/2.parquet], [/testdata/tpch/plan_sf0.02/supplier/3.parquet, /testdata/tpch/plan_sf0.02/supplier/4.parquet, /testdata/tpch/plan_sf0.02/supplier/5.parquet], [/testdata/tpch/plan_sf0.02/supplier/6.parquet, /testdata/tpch/plan_sf0.02/supplier/7.parquet, /testdata/tpch/plan_sf0.02/supplier/8.parquet], ...]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], file_type=parquet, predicate=DynamicFilter [ empty ] + │ 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 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/partsupp/1.parquet, /testdata/tpch/plan_sf0.02/partsupp/10.parquet, /testdata/tpch/plan_sf0.02/partsupp/11.parquet], [/testdata/tpch/plan_sf0.02/partsupp/12.parquet, /testdata/tpch/plan_sf0.02/partsupp/13.parquet, /testdata/tpch/plan_sf0.02/partsupp/14.parquet], [/testdata/tpch/plan_sf0.02/partsupp/15.parquet, /testdata/tpch/plan_sf0.02/partsupp/16.parquet, /testdata/tpch/plan_sf0.02/partsupp/2.parquet], [/testdata/tpch/plan_sf0.02/partsupp/3.parquet, /testdata/tpch/plan_sf0.02/partsupp/4.parquet, /testdata/tpch/plan_sf0.02/partsupp/5.parquet], [/testdata/tpch/plan_sf0.02/partsupp/6.parquet, /testdata/tpch/plan_sf0.02/partsupp/7.parquet, /testdata/tpch/plan_sf0.02/partsupp/8.parquet], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 @@ -94,30 +93,31 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/part/1.parquet, /testdata/tpch/plan_sf0.02/part/10.parquet, /testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet, /testdata/tpch/plan_sf0.02/part/13.parquet, /testdata/tpch/plan_sf0.02/part/14.parquet], [/testdata/tpch/plan_sf0.02/part/15.parquet, /testdata/tpch/plan_sf0.02/part/16.parquet, /testdata/tpch/plan_sf0.02/part/2.parquet], [/testdata/tpch/plan_sf0.02/part/3.parquet, /testdata/tpch/plan_sf0.02/part/4.parquet, /testdata/tpch/plan_sf0.02/part/5.parquet], [/testdata/tpch/plan_sf0.02/part/6.parquet, /testdata/tpch/plan_sf0.02/part/7.parquet, /testdata/tpch/plan_sf0.02/part/8.parquet], ...]}, projection=[p_partkey, p_mfgr, p_type, p_size], file_type=parquet, predicate=p_size@3 = 15 AND p_type@2 LIKE %BRASS, pruning_predicate=p_size_null_count@2 != row_count@3 AND p_size_min@0 <= 15 AND 15 <= p_size_max@1, required_guarantees=[p_size in (15)] └────────────────────────────────────────────────── ┌───── Stage 6 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] - │ RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 24), input_partitions=6 - │ ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] - │ AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] - │ CoalesceBatchesExec: target_batch_size=8192 + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 24), input_partitions=6 + │ ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] + │ AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p17] - │ RepartitionExec: partitioning=Hash([ps_partkey@0], 18), input_partitions=6 - │ AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] - │ CoalesceBatchesExec: target_batch_size=8192 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@2)], projection=[ps_partkey@1, ps_supplycost@2] - │ CoalescePartitionsExec - │ [Stage 4] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 - │ ProjectionExec: expr=[ps_partkey@1 as ps_partkey, ps_supplycost@2 as ps_supplycost, n_regionkey@0 as n_regionkey] - │ CoalesceBatchesExec: target_batch_size=8192 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_regionkey@1, ps_partkey@2, ps_supplycost@3] - │ CoalescePartitionsExec - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet, /testdata/tpch/plan_sf0.02/nation/10.parquet, /testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet, /testdata/tpch/plan_sf0.02/nation/13.parquet, /testdata/tpch/plan_sf0.02/nation/14.parquet], [/testdata/tpch/plan_sf0.02/nation/15.parquet, /testdata/tpch/plan_sf0.02/nation/16.parquet, /testdata/tpch/plan_sf0.02/nation/2.parquet], [/testdata/tpch/plan_sf0.02/nation/3.parquet, /testdata/tpch/plan_sf0.02/nation/4.parquet, /testdata/tpch/plan_sf0.02/nation/5.parquet], [/testdata/tpch/plan_sf0.02/nation/6.parquet, /testdata/tpch/plan_sf0.02/nation/7.parquet, /testdata/tpch/plan_sf0.02/nation/8.parquet], ...]}, projection=[n_nationkey, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ] - │ 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={6 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet, /testdata/tpch/plan_sf0.02/supplier/10.parquet, /testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet, /testdata/tpch/plan_sf0.02/supplier/13.parquet, /testdata/tpch/plan_sf0.02/supplier/14.parquet], [/testdata/tpch/plan_sf0.02/supplier/15.parquet, /testdata/tpch/plan_sf0.02/supplier/16.parquet, /testdata/tpch/plan_sf0.02/supplier/2.parquet], [/testdata/tpch/plan_sf0.02/supplier/3.parquet, /testdata/tpch/plan_sf0.02/supplier/4.parquet, /testdata/tpch/plan_sf0.02/supplier/5.parquet], [/testdata/tpch/plan_sf0.02/supplier/6.parquet, /testdata/tpch/plan_sf0.02/supplier/7.parquet, /testdata/tpch/plan_sf0.02/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/partsupp/1.parquet, /testdata/tpch/plan_sf0.02/partsupp/10.parquet, /testdata/tpch/plan_sf0.02/partsupp/11.parquet], [/testdata/tpch/plan_sf0.02/partsupp/12.parquet, /testdata/tpch/plan_sf0.02/partsupp/13.parquet, /testdata/tpch/plan_sf0.02/partsupp/14.parquet], [/testdata/tpch/plan_sf0.02/partsupp/15.parquet, /testdata/tpch/plan_sf0.02/partsupp/16.parquet, /testdata/tpch/plan_sf0.02/partsupp/2.parquet], [/testdata/tpch/plan_sf0.02/partsupp/3.parquet, /testdata/tpch/plan_sf0.02/partsupp/4.parquet, /testdata/tpch/plan_sf0.02/partsupp/5.parquet], [/testdata/tpch/plan_sf0.02/partsupp/6.parquet, /testdata/tpch/plan_sf0.02/partsupp/7.parquet, /testdata/tpch/plan_sf0.02/partsupp/8.parquet], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ empty ] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([ps_partkey@0], 18), input_partitions=6 + │ AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@2)], projection=[ps_partkey@1, ps_supplycost@2] + │ CoalescePartitionsExec + │ [Stage 4] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[ps_partkey@1 as ps_partkey, ps_supplycost@2 as ps_supplycost, n_regionkey@0 as n_regionkey] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_regionkey@1, ps_partkey@2, ps_supplycost@3] + │ CoalescePartitionsExec + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet, /testdata/tpch/plan_sf0.02/nation/10.parquet, /testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet, /testdata/tpch/plan_sf0.02/nation/13.parquet, /testdata/tpch/plan_sf0.02/nation/14.parquet], [/testdata/tpch/plan_sf0.02/nation/15.parquet, /testdata/tpch/plan_sf0.02/nation/16.parquet, /testdata/tpch/plan_sf0.02/nation/2.parquet], [/testdata/tpch/plan_sf0.02/nation/3.parquet, /testdata/tpch/plan_sf0.02/nation/4.parquet, /testdata/tpch/plan_sf0.02/nation/5.parquet], [/testdata/tpch/plan_sf0.02/nation/6.parquet, /testdata/tpch/plan_sf0.02/nation/7.parquet, /testdata/tpch/plan_sf0.02/nation/8.parquet], ...]}, projection=[n_nationkey, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ] + │ 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={6 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet, /testdata/tpch/plan_sf0.02/supplier/10.parquet, /testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet, /testdata/tpch/plan_sf0.02/supplier/13.parquet, /testdata/tpch/plan_sf0.02/supplier/14.parquet], [/testdata/tpch/plan_sf0.02/supplier/15.parquet, /testdata/tpch/plan_sf0.02/supplier/16.parquet, /testdata/tpch/plan_sf0.02/supplier/2.parquet], [/testdata/tpch/plan_sf0.02/supplier/3.parquet, /testdata/tpch/plan_sf0.02/supplier/4.parquet, /testdata/tpch/plan_sf0.02/supplier/5.parquet], [/testdata/tpch/plan_sf0.02/supplier/6.parquet, /testdata/tpch/plan_sf0.02/supplier/7.parquet, /testdata/tpch/plan_sf0.02/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/partsupp/1.parquet, /testdata/tpch/plan_sf0.02/partsupp/10.parquet, /testdata/tpch/plan_sf0.02/partsupp/11.parquet], [/testdata/tpch/plan_sf0.02/partsupp/12.parquet, /testdata/tpch/plan_sf0.02/partsupp/13.parquet, /testdata/tpch/plan_sf0.02/partsupp/14.parquet], [/testdata/tpch/plan_sf0.02/partsupp/15.parquet, /testdata/tpch/plan_sf0.02/partsupp/16.parquet, /testdata/tpch/plan_sf0.02/partsupp/2.parquet], [/testdata/tpch/plan_sf0.02/partsupp/3.parquet, /testdata/tpch/plan_sf0.02/partsupp/4.parquet, /testdata/tpch/plan_sf0.02/partsupp/5.parquet], [/testdata/tpch/plan_sf0.02/partsupp/6.parquet, /testdata/tpch/plan_sf0.02/partsupp/7.parquet, /testdata/tpch/plan_sf0.02/partsupp/8.parquet], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 @@ -141,25 +141,25 @@ mod tests { │ 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 - │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1 + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p17] - │ RepartitionExec: partitioning=Hash([l_orderkey@0, o_orderdate@1, o_shippriority@2], 18), input_partitions=6 - │ AggregateExec: mode=Partial, gby=[l_orderkey@2 as l_orderkey, o_orderdate@0 as o_orderdate, o_shippriority@1 as o_shippriority], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - │ CoalesceBatchesExec: target_batch_size=8192 - │ 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 + │ RepartitionExec: partitioning=Hash([l_orderkey@0, o_orderdate@1, o_shippriority@2], 18), input_partitions=6 + │ AggregateExec: mode=Partial, gby=[l_orderkey@2 as l_orderkey, o_orderdate@0 as o_orderdate, o_shippriority@1 as o_shippriority], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ 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 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: o_orderdate@2 < 1995-03-15 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet, /testdata/tpch/plan_sf0.02/orders/10.parquet, /testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet, /testdata/tpch/plan_sf0.02/orders/13.parquet, /testdata/tpch/plan_sf0.02/orders/14.parquet], [/testdata/tpch/plan_sf0.02/orders/15.parquet, /testdata/tpch/plan_sf0.02/orders/16.parquet, /testdata/tpch/plan_sf0.02/orders/2.parquet], [/testdata/tpch/plan_sf0.02/orders/3.parquet, /testdata/tpch/plan_sf0.02/orders/4.parquet, /testdata/tpch/plan_sf0.02/orders/5.parquet], [/testdata/tpch/plan_sf0.02/orders/6.parquet, /testdata/tpch/plan_sf0.02/orders/7.parquet, /testdata/tpch/plan_sf0.02/orders/8.parquet], ...]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], file_type=parquet, predicate=o_orderdate@2 < 1995-03-15 AND DynamicFilter [ empty ], pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@0 < 1995-03-15, required_guarantees=[] │ 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 - │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: o_orderdate@2 < 1995-03-15 - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet, /testdata/tpch/plan_sf0.02/orders/10.parquet, /testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet, /testdata/tpch/plan_sf0.02/orders/13.parquet, /testdata/tpch/plan_sf0.02/orders/14.parquet], [/testdata/tpch/plan_sf0.02/orders/15.parquet, /testdata/tpch/plan_sf0.02/orders/16.parquet, /testdata/tpch/plan_sf0.02/orders/2.parquet], [/testdata/tpch/plan_sf0.02/orders/3.parquet, /testdata/tpch/plan_sf0.02/orders/4.parquet, /testdata/tpch/plan_sf0.02/orders/5.parquet], [/testdata/tpch/plan_sf0.02/orders/6.parquet, /testdata/tpch/plan_sf0.02/orders/7.parquet, /testdata/tpch/plan_sf0.02/orders/8.parquet], ...]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], file_type=parquet, predicate=o_orderdate@2 < 1995-03-15 AND DynamicFilter [ empty ], pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@0 < 1995-03-15, required_guarantees=[] - │ 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={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 > 1995-03-15 AND DynamicFilter [ empty ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 > 1995-03-15, required_guarantees=[] + │ FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 > 1995-03-15 AND DynamicFilter [ empty ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 > 1995-03-15, required_guarantees=[] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 @@ -183,19 +183,19 @@ mod tests { │ 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 - │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1 + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p17] - │ RepartitionExec: partitioning=Hash([o_orderpriority@0], 18), input_partitions=6 - │ AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))] - │ CoalesceBatchesExec: target_batch_size=8192 - │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(l_orderkey@0, o_orderkey@0)], projection=[o_orderpriority@1] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet, /testdata/tpch/plan_sf0.02/orders/10.parquet, /testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet, /testdata/tpch/plan_sf0.02/orders/13.parquet, /testdata/tpch/plan_sf0.02/orders/14.parquet], [/testdata/tpch/plan_sf0.02/orders/15.parquet, /testdata/tpch/plan_sf0.02/orders/16.parquet, /testdata/tpch/plan_sf0.02/orders/2.parquet], [/testdata/tpch/plan_sf0.02/orders/3.parquet, /testdata/tpch/plan_sf0.02/orders/4.parquet, /testdata/tpch/plan_sf0.02/orders/5.parquet], [/testdata/tpch/plan_sf0.02/orders/6.parquet, /testdata/tpch/plan_sf0.02/orders/7.parquet, /testdata/tpch/plan_sf0.02/orders/8.parquet], ...]}, projection=[o_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=[] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([o_orderpriority@0], 18), input_partitions=6 + │ AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(l_orderkey@0, o_orderkey@0)], projection=[o_orderpriority@1] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet, /testdata/tpch/plan_sf0.02/orders/10.parquet, /testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet, /testdata/tpch/plan_sf0.02/orders/13.parquet, /testdata/tpch/plan_sf0.02/orders/14.parquet], [/testdata/tpch/plan_sf0.02/orders/15.parquet, /testdata/tpch/plan_sf0.02/orders/16.parquet, /testdata/tpch/plan_sf0.02/orders/2.parquet], [/testdata/tpch/plan_sf0.02/orders/3.parquet, /testdata/tpch/plan_sf0.02/orders/4.parquet, /testdata/tpch/plan_sf0.02/orders/5.parquet], [/testdata/tpch/plan_sf0.02/orders/6.parquet, /testdata/tpch/plan_sf0.02/orders/7.parquet, /testdata/tpch/plan_sf0.02/orders/8.parquet], ...]}, projection=[o_orderkey, o_orderdate, o_orderpriority], file_type=parquet, predicate=o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1993-07-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 < 1993-10-01, required_guarantees=[] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 @@ -219,36 +219,36 @@ mod tests { │ 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 - │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p17] - │ RepartitionExec: partitioning=Hash([n_name@0], 18), input_partitions=6 - │ AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - │ CoalesceBatchesExec: target_batch_size=8192 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@3)], projection=[l_extendedprice@1, l_discount@2, n_name@3] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 - │ ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, n_name@0 as n_name, n_regionkey@1 as n_regionkey] - │ CoalesceBatchesExec: target_batch_size=8192 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_name@1, n_regionkey@2, l_extendedprice@3, l_discount@4] - │ CoalescePartitionsExec - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet, /testdata/tpch/plan_sf0.02/nation/10.parquet, /testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet, /testdata/tpch/plan_sf0.02/nation/13.parquet, /testdata/tpch/plan_sf0.02/nation/14.parquet], [/testdata/tpch/plan_sf0.02/nation/15.parquet, /testdata/tpch/plan_sf0.02/nation/16.parquet, /testdata/tpch/plan_sf0.02/nation/2.parquet], [/testdata/tpch/plan_sf0.02/nation/3.parquet, /testdata/tpch/plan_sf0.02/nation/4.parquet, /testdata/tpch/plan_sf0.02/nation/5.parquet], [/testdata/tpch/plan_sf0.02/nation/6.parquet, /testdata/tpch/plan_sf0.02/nation/7.parquet, /testdata/tpch/plan_sf0.02/nation/8.parquet], ...]}, projection=[n_nationkey, n_name, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ] - │ 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={6 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet, /testdata/tpch/plan_sf0.02/supplier/10.parquet, /testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet, /testdata/tpch/plan_sf0.02/supplier/13.parquet, /testdata/tpch/plan_sf0.02/supplier/14.parquet], [/testdata/tpch/plan_sf0.02/supplier/15.parquet, /testdata/tpch/plan_sf0.02/supplier/16.parquet, /testdata/tpch/plan_sf0.02/supplier/2.parquet], [/testdata/tpch/plan_sf0.02/supplier/3.parquet, /testdata/tpch/plan_sf0.02/supplier/4.parquet, /testdata/tpch/plan_sf0.02/supplier/5.parquet], [/testdata/tpch/plan_sf0.02/supplier/6.parquet, /testdata/tpch/plan_sf0.02/supplier/7.parquet, /testdata/tpch/plan_sf0.02/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] - │ 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 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/customer/1.parquet, /testdata/tpch/plan_sf0.02/customer/10.parquet, /testdata/tpch/plan_sf0.02/customer/11.parquet], [/testdata/tpch/plan_sf0.02/customer/12.parquet, /testdata/tpch/plan_sf0.02/customer/13.parquet, /testdata/tpch/plan_sf0.02/customer/14.parquet], [/testdata/tpch/plan_sf0.02/customer/15.parquet, /testdata/tpch/plan_sf0.02/customer/16.parquet, /testdata/tpch/plan_sf0.02/customer/2.parquet], [/testdata/tpch/plan_sf0.02/customer/3.parquet, /testdata/tpch/plan_sf0.02/customer/4.parquet, /testdata/tpch/plan_sf0.02/customer/5.parquet], [/testdata/tpch/plan_sf0.02/customer/6.parquet, /testdata/tpch/plan_sf0.02/customer/7.parquet, /testdata/tpch/plan_sf0.02/customer/8.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilter [ empty ] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([n_name@0], 18), input_partitions=6 + │ AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@3)], projection=[l_extendedprice@1, l_discount@2, n_name@3] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, n_name@0 as n_name, n_regionkey@1 as n_regionkey] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_name@1, n_regionkey@2, l_extendedprice@3, l_discount@4] + │ CoalescePartitionsExec + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet, /testdata/tpch/plan_sf0.02/nation/10.parquet, /testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet, /testdata/tpch/plan_sf0.02/nation/13.parquet, /testdata/tpch/plan_sf0.02/nation/14.parquet], [/testdata/tpch/plan_sf0.02/nation/15.parquet, /testdata/tpch/plan_sf0.02/nation/16.parquet, /testdata/tpch/plan_sf0.02/nation/2.parquet], [/testdata/tpch/plan_sf0.02/nation/3.parquet, /testdata/tpch/plan_sf0.02/nation/4.parquet, /testdata/tpch/plan_sf0.02/nation/5.parquet], [/testdata/tpch/plan_sf0.02/nation/6.parquet, /testdata/tpch/plan_sf0.02/nation/7.parquet, /testdata/tpch/plan_sf0.02/nation/8.parquet], ...]}, projection=[n_nationkey, n_name, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ] + │ 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={6 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet, /testdata/tpch/plan_sf0.02/supplier/10.parquet, /testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet, /testdata/tpch/plan_sf0.02/supplier/13.parquet, /testdata/tpch/plan_sf0.02/supplier/14.parquet], [/testdata/tpch/plan_sf0.02/supplier/15.parquet, /testdata/tpch/plan_sf0.02/supplier/16.parquet, /testdata/tpch/plan_sf0.02/supplier/2.parquet], [/testdata/tpch/plan_sf0.02/supplier/3.parquet, /testdata/tpch/plan_sf0.02/supplier/4.parquet, /testdata/tpch/plan_sf0.02/supplier/5.parquet], [/testdata/tpch/plan_sf0.02/supplier/6.parquet, /testdata/tpch/plan_sf0.02/supplier/7.parquet, /testdata/tpch/plan_sf0.02/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + │ 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 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/customer/1.parquet, /testdata/tpch/plan_sf0.02/customer/10.parquet, /testdata/tpch/plan_sf0.02/customer/11.parquet], [/testdata/tpch/plan_sf0.02/customer/12.parquet, /testdata/tpch/plan_sf0.02/customer/13.parquet, /testdata/tpch/plan_sf0.02/customer/14.parquet], [/testdata/tpch/plan_sf0.02/customer/15.parquet, /testdata/tpch/plan_sf0.02/customer/16.parquet, /testdata/tpch/plan_sf0.02/customer/2.parquet], [/testdata/tpch/plan_sf0.02/customer/3.parquet, /testdata/tpch/plan_sf0.02/customer/4.parquet, /testdata/tpch/plan_sf0.02/customer/5.parquet], [/testdata/tpch/plan_sf0.02/customer/6.parquet, /testdata/tpch/plan_sf0.02/customer/7.parquet, /testdata/tpch/plan_sf0.02/customer/8.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 @@ -299,39 +299,38 @@ mod tests { │ 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 - │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=1 + │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p17] - │ RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 18), input_partitions=6 - │ AggregateExec: mode=Partial, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)] - │ ProjectionExec: expr=[n_name@4 as supp_nation, n_name@0 as cust_nation, date_part(YEAR, l_shipdate@3) as l_year, l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as volume] - │ CoalesceBatchesExec: target_batch_size=8192 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, c_nationkey@3)], filter=n_name@0 = FRANCE AND n_name@1 = GERMANY OR n_name@0 = GERMANY AND n_name@1 = FRANCE, projection=[n_name@1, l_extendedprice@2, l_discount@3, l_shipdate@4, n_name@6] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 - │ ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_shipdate@3 as l_shipdate, c_nationkey@4 as c_nationkey, n_name@0 as n_name] - │ CoalesceBatchesExec: target_batch_size=8192 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@0)], projection=[n_name@1, l_extendedprice@3, l_discount@4, l_shipdate@5, c_nationkey@6] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 - │ ProjectionExec: expr=[s_nationkey@1 as s_nationkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, l_shipdate@4 as l_shipdate, c_nationkey@0 as c_nationkey] - │ CoalesceBatchesExec: target_batch_size=8192 - │ HashJoinExec: mode=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={6 groups: [[/testdata/tpch/plan_sf0.02/customer/1.parquet, /testdata/tpch/plan_sf0.02/customer/10.parquet, /testdata/tpch/plan_sf0.02/customer/11.parquet], [/testdata/tpch/plan_sf0.02/customer/12.parquet, /testdata/tpch/plan_sf0.02/customer/13.parquet, /testdata/tpch/plan_sf0.02/customer/14.parquet], [/testdata/tpch/plan_sf0.02/customer/15.parquet, /testdata/tpch/plan_sf0.02/customer/16.parquet, /testdata/tpch/plan_sf0.02/customer/2.parquet], [/testdata/tpch/plan_sf0.02/customer/3.parquet, /testdata/tpch/plan_sf0.02/customer/4.parquet, /testdata/tpch/plan_sf0.02/customer/5.parquet], [/testdata/tpch/plan_sf0.02/customer/6.parquet, /testdata/tpch/plan_sf0.02/customer/7.parquet, /testdata/tpch/plan_sf0.02/customer/8.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] - │ CoalesceBatchesExec: target_batch_size=8192 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@1, o_orderkey@0)], projection=[s_nationkey@0, l_extendedprice@2, l_discount@3, l_shipdate@4, o_custkey@6] - │ CoalesceBatchesExec: target_batch_size=8192 - │ RepartitionExec: partitioning=Hash([l_orderkey@1], 6), input_partitions=6 - │ CoalesceBatchesExec: target_batch_size=8192 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5, l_shipdate@6] - │ CoalescePartitionsExec - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet, /testdata/tpch/plan_sf0.02/supplier/10.parquet, /testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet, /testdata/tpch/plan_sf0.02/supplier/13.parquet, /testdata/tpch/plan_sf0.02/supplier/14.parquet], [/testdata/tpch/plan_sf0.02/supplier/15.parquet, /testdata/tpch/plan_sf0.02/supplier/16.parquet, /testdata/tpch/plan_sf0.02/supplier/2.parquet], [/testdata/tpch/plan_sf0.02/supplier/3.parquet, /testdata/tpch/plan_sf0.02/supplier/4.parquet, /testdata/tpch/plan_sf0.02/supplier/5.parquet], [/testdata/tpch/plan_sf0.02/supplier/6.parquet, /testdata/tpch/plan_sf0.02/supplier/7.parquet, /testdata/tpch/plan_sf0.02/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 AND DynamicFilter [ empty ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1995-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 <= 1996-12-31, required_guarantees=[] - │ CoalesceBatchesExec: target_batch_size=8192 + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 18), input_partitions=6 + │ AggregateExec: mode=Partial, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)] + │ ProjectionExec: expr=[n_name@4 as supp_nation, n_name@0 as cust_nation, date_part(YEAR, l_shipdate@3) as l_year, l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as volume] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, c_nationkey@3)], filter=n_name@0 = FRANCE AND n_name@1 = GERMANY OR n_name@0 = GERMANY AND n_name@1 = FRANCE, projection=[n_name@1, l_extendedprice@2, l_discount@3, l_shipdate@4, n_name@6] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_shipdate@3 as l_shipdate, c_nationkey@4 as c_nationkey, n_name@0 as n_name] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@0)], projection=[n_name@1, l_extendedprice@3, l_discount@4, l_shipdate@5, c_nationkey@6] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[s_nationkey@1 as s_nationkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, l_shipdate@4 as l_shipdate, c_nationkey@0 as c_nationkey] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=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={6 groups: [[/testdata/tpch/plan_sf0.02/customer/1.parquet, /testdata/tpch/plan_sf0.02/customer/10.parquet, /testdata/tpch/plan_sf0.02/customer/11.parquet], [/testdata/tpch/plan_sf0.02/customer/12.parquet, /testdata/tpch/plan_sf0.02/customer/13.parquet, /testdata/tpch/plan_sf0.02/customer/14.parquet], [/testdata/tpch/plan_sf0.02/customer/15.parquet, /testdata/tpch/plan_sf0.02/customer/16.parquet, /testdata/tpch/plan_sf0.02/customer/2.parquet], [/testdata/tpch/plan_sf0.02/customer/3.parquet, /testdata/tpch/plan_sf0.02/customer/4.parquet, /testdata/tpch/plan_sf0.02/customer/5.parquet], [/testdata/tpch/plan_sf0.02/customer/6.parquet, /testdata/tpch/plan_sf0.02/customer/7.parquet, /testdata/tpch/plan_sf0.02/customer/8.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@1, o_orderkey@0)], projection=[s_nationkey@0, l_extendedprice@2, l_discount@3, l_shipdate@4, o_custkey@6] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([l_orderkey@1], 6), input_partitions=6 + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5, l_shipdate@6] + │ CoalescePartitionsExec + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet, /testdata/tpch/plan_sf0.02/supplier/10.parquet, /testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet, /testdata/tpch/plan_sf0.02/supplier/13.parquet, /testdata/tpch/plan_sf0.02/supplier/14.parquet], [/testdata/tpch/plan_sf0.02/supplier/15.parquet, /testdata/tpch/plan_sf0.02/supplier/16.parquet, /testdata/tpch/plan_sf0.02/supplier/2.parquet], [/testdata/tpch/plan_sf0.02/supplier/3.parquet, /testdata/tpch/plan_sf0.02/supplier/4.parquet, /testdata/tpch/plan_sf0.02/supplier/5.parquet], [/testdata/tpch/plan_sf0.02/supplier/6.parquet, /testdata/tpch/plan_sf0.02/supplier/7.parquet, /testdata/tpch/plan_sf0.02/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 AND DynamicFilter [ empty ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1995-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 <= 1996-12-31, required_guarantees=[] │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] @@ -347,9 +346,10 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet, /testdata/tpch/plan_sf0.02/nation/10.parquet, /testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet, /testdata/tpch/plan_sf0.02/nation/13.parquet, /testdata/tpch/plan_sf0.02/nation/14.parquet], [/testdata/tpch/plan_sf0.02/nation/15.parquet, /testdata/tpch/plan_sf0.02/nation/16.parquet, /testdata/tpch/plan_sf0.02/nation/2.parquet], [/testdata/tpch/plan_sf0.02/nation/3.parquet, /testdata/tpch/plan_sf0.02/nation/4.parquet, /testdata/tpch/plan_sf0.02/nation/5.parquet], [/testdata/tpch/plan_sf0.02/nation/6.parquet, /testdata/tpch/plan_sf0.02/nation/7.parquet, /testdata/tpch/plan_sf0.02/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = FRANCE OR n_name@1 = GERMANY, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= FRANCE AND FRANCE <= n_name_max@1 OR n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1, required_guarantees=[n_name in (FRANCE, GERMANY)] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ RepartitionExec: partitioning=Hash([o_orderkey@0], 6), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet, /testdata/tpch/plan_sf0.02/orders/10.parquet, /testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet, /testdata/tpch/plan_sf0.02/orders/13.parquet, /testdata/tpch/plan_sf0.02/orders/14.parquet], [/testdata/tpch/plan_sf0.02/orders/15.parquet, /testdata/tpch/plan_sf0.02/orders/16.parquet, /testdata/tpch/plan_sf0.02/orders/2.parquet], [/testdata/tpch/plan_sf0.02/orders/3.parquet, /testdata/tpch/plan_sf0.02/orders/4.parquet, /testdata/tpch/plan_sf0.02/orders/5.parquet], [/testdata/tpch/plan_sf0.02/orders/6.parquet, /testdata/tpch/plan_sf0.02/orders/7.parquet, /testdata/tpch/plan_sf0.02/orders/8.parquet], ...]}, projection=[o_orderkey, o_custkey], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 6), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet, /testdata/tpch/plan_sf0.02/orders/10.parquet, /testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet, /testdata/tpch/plan_sf0.02/orders/13.parquet, /testdata/tpch/plan_sf0.02/orders/14.parquet], [/testdata/tpch/plan_sf0.02/orders/15.parquet, /testdata/tpch/plan_sf0.02/orders/16.parquet, /testdata/tpch/plan_sf0.02/orders/2.parquet], [/testdata/tpch/plan_sf0.02/orders/3.parquet, /testdata/tpch/plan_sf0.02/orders/4.parquet, /testdata/tpch/plan_sf0.02/orders/5.parquet], [/testdata/tpch/plan_sf0.02/orders/6.parquet, /testdata/tpch/plan_sf0.02/orders/7.parquet, /testdata/tpch/plan_sf0.02/orders/8.parquet], ...]}, projection=[o_orderkey, o_custkey], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── "); Ok(()) @@ -367,46 +367,46 @@ mod tests { │ 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 - │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=1 + │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p17] - │ RepartitionExec: partitioning=Hash([o_year@0], 18), input_partitions=6 - │ AggregateExec: mode=Partial, gby=[o_year@0 as o_year], aggr=[sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)] - │ ProjectionExec: expr=[date_part(YEAR, o_orderdate@2) as o_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume, n_name@3 as nation] - │ CoalesceBatchesExec: target_batch_size=8192 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@3)], projection=[l_extendedprice@1, l_discount@2, o_orderdate@3, n_name@5] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 - │ ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, o_orderdate@3 as o_orderdate, n_regionkey@4 as n_regionkey, n_name@0 as n_name] - │ CoalesceBatchesExec: target_batch_size=8192 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_name@1, l_extendedprice@2, l_discount@3, o_orderdate@5, n_regionkey@6] - │ CoalescePartitionsExec - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet, /testdata/tpch/plan_sf0.02/nation/10.parquet, /testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet, /testdata/tpch/plan_sf0.02/nation/13.parquet, /testdata/tpch/plan_sf0.02/nation/14.parquet], [/testdata/tpch/plan_sf0.02/nation/15.parquet, /testdata/tpch/plan_sf0.02/nation/16.parquet, /testdata/tpch/plan_sf0.02/nation/2.parquet], [/testdata/tpch/plan_sf0.02/nation/3.parquet, /testdata/tpch/plan_sf0.02/nation/4.parquet, /testdata/tpch/plan_sf0.02/nation/5.parquet], [/testdata/tpch/plan_sf0.02/nation/6.parquet, /testdata/tpch/plan_sf0.02/nation/7.parquet, /testdata/tpch/plan_sf0.02/nation/8.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={6 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet, /testdata/tpch/plan_sf0.02/nation/10.parquet, /testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet, /testdata/tpch/plan_sf0.02/nation/13.parquet, /testdata/tpch/plan_sf0.02/nation/14.parquet], [/testdata/tpch/plan_sf0.02/nation/15.parquet, /testdata/tpch/plan_sf0.02/nation/16.parquet, /testdata/tpch/plan_sf0.02/nation/2.parquet], [/testdata/tpch/plan_sf0.02/nation/3.parquet, /testdata/tpch/plan_sf0.02/nation/4.parquet, /testdata/tpch/plan_sf0.02/nation/5.parquet], [/testdata/tpch/plan_sf0.02/nation/6.parquet, /testdata/tpch/plan_sf0.02/nation/7.parquet, /testdata/tpch/plan_sf0.02/nation/8.parquet], ...]}, projection=[n_nationkey, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ] - │ 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 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 - │ ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@0 as s_nationkey] - │ 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={6 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet, /testdata/tpch/plan_sf0.02/supplier/10.parquet, /testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet, /testdata/tpch/plan_sf0.02/supplier/13.parquet, /testdata/tpch/plan_sf0.02/supplier/14.parquet], [/testdata/tpch/plan_sf0.02/supplier/15.parquet, /testdata/tpch/plan_sf0.02/supplier/16.parquet, /testdata/tpch/plan_sf0.02/supplier/2.parquet], [/testdata/tpch/plan_sf0.02/supplier/3.parquet, /testdata/tpch/plan_sf0.02/supplier/4.parquet, /testdata/tpch/plan_sf0.02/supplier/5.parquet], [/testdata/tpch/plan_sf0.02/supplier/6.parquet, /testdata/tpch/plan_sf0.02/supplier/7.parquet, /testdata/tpch/plan_sf0.02/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] - │ 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 - │ [Stage 3] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/customer/1.parquet, /testdata/tpch/plan_sf0.02/customer/10.parquet, /testdata/tpch/plan_sf0.02/customer/11.parquet], [/testdata/tpch/plan_sf0.02/customer/12.parquet, /testdata/tpch/plan_sf0.02/customer/13.parquet, /testdata/tpch/plan_sf0.02/customer/14.parquet], [/testdata/tpch/plan_sf0.02/customer/15.parquet, /testdata/tpch/plan_sf0.02/customer/16.parquet, /testdata/tpch/plan_sf0.02/customer/2.parquet], [/testdata/tpch/plan_sf0.02/customer/3.parquet, /testdata/tpch/plan_sf0.02/customer/4.parquet, /testdata/tpch/plan_sf0.02/customer/5.parquet], [/testdata/tpch/plan_sf0.02/customer/6.parquet, /testdata/tpch/plan_sf0.02/customer/7.parquet, /testdata/tpch/plan_sf0.02/customer/8.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([o_year@0], 18), input_partitions=6 + │ AggregateExec: mode=Partial, gby=[o_year@0 as o_year], aggr=[sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)] + │ ProjectionExec: expr=[date_part(YEAR, o_orderdate@2) as o_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume, n_name@3 as nation] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@3)], projection=[l_extendedprice@1, l_discount@2, o_orderdate@3, n_name@5] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, o_orderdate@3 as o_orderdate, n_regionkey@4 as n_regionkey, n_name@0 as n_name] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_name@1, l_extendedprice@2, l_discount@3, o_orderdate@5, n_regionkey@6] + │ CoalescePartitionsExec + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet, /testdata/tpch/plan_sf0.02/nation/10.parquet, /testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet, /testdata/tpch/plan_sf0.02/nation/13.parquet, /testdata/tpch/plan_sf0.02/nation/14.parquet], [/testdata/tpch/plan_sf0.02/nation/15.parquet, /testdata/tpch/plan_sf0.02/nation/16.parquet, /testdata/tpch/plan_sf0.02/nation/2.parquet], [/testdata/tpch/plan_sf0.02/nation/3.parquet, /testdata/tpch/plan_sf0.02/nation/4.parquet, /testdata/tpch/plan_sf0.02/nation/5.parquet], [/testdata/tpch/plan_sf0.02/nation/6.parquet, /testdata/tpch/plan_sf0.02/nation/7.parquet, /testdata/tpch/plan_sf0.02/nation/8.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={6 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet, /testdata/tpch/plan_sf0.02/nation/10.parquet, /testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet, /testdata/tpch/plan_sf0.02/nation/13.parquet, /testdata/tpch/plan_sf0.02/nation/14.parquet], [/testdata/tpch/plan_sf0.02/nation/15.parquet, /testdata/tpch/plan_sf0.02/nation/16.parquet, /testdata/tpch/plan_sf0.02/nation/2.parquet], [/testdata/tpch/plan_sf0.02/nation/3.parquet, /testdata/tpch/plan_sf0.02/nation/4.parquet, /testdata/tpch/plan_sf0.02/nation/5.parquet], [/testdata/tpch/plan_sf0.02/nation/6.parquet, /testdata/tpch/plan_sf0.02/nation/7.parquet, /testdata/tpch/plan_sf0.02/nation/8.parquet], ...]}, projection=[n_nationkey, n_regionkey], file_type=parquet, predicate=DynamicFilter [ empty ] + │ 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 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@0 as s_nationkey] + │ 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={6 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet, /testdata/tpch/plan_sf0.02/supplier/10.parquet, /testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet, /testdata/tpch/plan_sf0.02/supplier/13.parquet, /testdata/tpch/plan_sf0.02/supplier/14.parquet], [/testdata/tpch/plan_sf0.02/supplier/15.parquet, /testdata/tpch/plan_sf0.02/supplier/16.parquet, /testdata/tpch/plan_sf0.02/supplier/2.parquet], [/testdata/tpch/plan_sf0.02/supplier/3.parquet, /testdata/tpch/plan_sf0.02/supplier/4.parquet, /testdata/tpch/plan_sf0.02/supplier/5.parquet], [/testdata/tpch/plan_sf0.02/supplier/6.parquet, /testdata/tpch/plan_sf0.02/supplier/7.parquet, /testdata/tpch/plan_sf0.02/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + │ 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 + │ [Stage 3] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/customer/1.parquet, /testdata/tpch/plan_sf0.02/customer/10.parquet, /testdata/tpch/plan_sf0.02/customer/11.parquet], [/testdata/tpch/plan_sf0.02/customer/12.parquet, /testdata/tpch/plan_sf0.02/customer/13.parquet, /testdata/tpch/plan_sf0.02/customer/14.parquet], [/testdata/tpch/plan_sf0.02/customer/15.parquet, /testdata/tpch/plan_sf0.02/customer/16.parquet, /testdata/tpch/plan_sf0.02/customer/2.parquet], [/testdata/tpch/plan_sf0.02/customer/3.parquet, /testdata/tpch/plan_sf0.02/customer/4.parquet, /testdata/tpch/plan_sf0.02/customer/5.parquet], [/testdata/tpch/plan_sf0.02/customer/6.parquet, /testdata/tpch/plan_sf0.02/customer/7.parquet, /testdata/tpch/plan_sf0.02/customer/8.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 @@ -442,51 +442,50 @@ mod tests { │ 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 - │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=1 + │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── ┌───── Stage 6 ── Tasks: t0:[p0..p17] - │ RepartitionExec: partitioning=Hash([nation@0, o_year@1], 18), input_partitions=6 - │ AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] - │ ProjectionExec: expr=[n_name@0 as nation, date_part(YEAR, o_orderdate@5) as o_year, l_extendedprice@2 * (Some(1),20,0 - l_discount@3) - ps_supplycost@4 * l_quantity@1 as amount] - │ 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={6 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet, /testdata/tpch/plan_sf0.02/nation/10.parquet, /testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet, /testdata/tpch/plan_sf0.02/nation/13.parquet, /testdata/tpch/plan_sf0.02/nation/14.parquet], [/testdata/tpch/plan_sf0.02/nation/15.parquet, /testdata/tpch/plan_sf0.02/nation/16.parquet, /testdata/tpch/plan_sf0.02/nation/2.parquet], [/testdata/tpch/plan_sf0.02/nation/3.parquet, /testdata/tpch/plan_sf0.02/nation/4.parquet, /testdata/tpch/plan_sf0.02/nation/5.parquet], [/testdata/tpch/plan_sf0.02/nation/6.parquet, /testdata/tpch/plan_sf0.02/nation/7.parquet, /testdata/tpch/plan_sf0.02/nation/8.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=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@7] - │ CoalesceBatchesExec: target_batch_size=8192 + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([nation@0, o_year@1], 18), input_partitions=6 + │ AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] + │ ProjectionExec: expr=[n_name@0 as nation, date_part(YEAR, o_orderdate@5) as o_year, l_extendedprice@2 * (Some(1),20,0 - l_discount@3) - ps_supplycost@4 * l_quantity@1 as amount] + │ 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={6 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet, /testdata/tpch/plan_sf0.02/nation/10.parquet, /testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet, /testdata/tpch/plan_sf0.02/nation/13.parquet, /testdata/tpch/plan_sf0.02/nation/14.parquet], [/testdata/tpch/plan_sf0.02/nation/15.parquet, /testdata/tpch/plan_sf0.02/nation/16.parquet, /testdata/tpch/plan_sf0.02/nation/2.parquet], [/testdata/tpch/plan_sf0.02/nation/3.parquet, /testdata/tpch/plan_sf0.02/nation/4.parquet, /testdata/tpch/plan_sf0.02/nation/5.parquet], [/testdata/tpch/plan_sf0.02/nation/6.parquet, /testdata/tpch/plan_sf0.02/nation/7.parquet, /testdata/tpch/plan_sf0.02/nation/8.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=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@7] │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4 - │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ RepartitionExec: partitioning=Hash([o_orderkey@0], 6), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet, /testdata/tpch/plan_sf0.02/orders/10.parquet, /testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet, /testdata/tpch/plan_sf0.02/orders/13.parquet, /testdata/tpch/plan_sf0.02/orders/14.parquet], [/testdata/tpch/plan_sf0.02/orders/15.parquet, /testdata/tpch/plan_sf0.02/orders/16.parquet, /testdata/tpch/plan_sf0.02/orders/2.parquet], [/testdata/tpch/plan_sf0.02/orders/3.parquet, /testdata/tpch/plan_sf0.02/orders/4.parquet, /testdata/tpch/plan_sf0.02/orders/5.parquet], [/testdata/tpch/plan_sf0.02/orders/6.parquet, /testdata/tpch/plan_sf0.02/orders/7.parquet, /testdata/tpch/plan_sf0.02/orders/8.parquet], ...]}, projection=[o_orderkey, o_orderdate], file_type=parquet + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 6), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet, /testdata/tpch/plan_sf0.02/orders/10.parquet, /testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet, /testdata/tpch/plan_sf0.02/orders/13.parquet, /testdata/tpch/plan_sf0.02/orders/14.parquet], [/testdata/tpch/plan_sf0.02/orders/15.parquet, /testdata/tpch/plan_sf0.02/orders/16.parquet, /testdata/tpch/plan_sf0.02/orders/2.parquet], [/testdata/tpch/plan_sf0.02/orders/3.parquet, /testdata/tpch/plan_sf0.02/orders/4.parquet, /testdata/tpch/plan_sf0.02/orders/5.parquet], [/testdata/tpch/plan_sf0.02/orders/6.parquet, /testdata/tpch/plan_sf0.02/orders/7.parquet, /testdata/tpch/plan_sf0.02/orders/8.parquet], ...]}, projection=[o_orderkey, o_orderdate], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ RepartitionExec: partitioning=Hash([l_orderkey@0], 6), input_partitions=6 - │ CoalesceBatchesExec: target_batch_size=8192 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, ps_suppkey@1), (l_partkey@1, ps_partkey@0)], projection=[l_orderkey@0, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@9] - │ CoalesceBatchesExec: target_batch_size=8192 + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 6), input_partitions=6 + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, ps_suppkey@1), (l_partkey@1, ps_partkey@0)], projection=[l_orderkey@0, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@9] │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1 - │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p23] - │ RepartitionExec: partitioning=Hash([l_suppkey@2, l_partkey@1], 24), input_partitions=6 - │ ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_partkey@2 as l_partkey, l_suppkey@3 as l_suppkey, l_quantity@4 as l_quantity, l_extendedprice@5 as l_extendedprice, l_discount@6 as l_discount, s_nationkey@0 as s_nationkey] - │ CoalesceBatchesExec: target_batch_size=8192 - │ 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={6 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet, /testdata/tpch/plan_sf0.02/supplier/10.parquet, /testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet, /testdata/tpch/plan_sf0.02/supplier/13.parquet, /testdata/tpch/plan_sf0.02/supplier/14.parquet], [/testdata/tpch/plan_sf0.02/supplier/15.parquet, /testdata/tpch/plan_sf0.02/supplier/16.parquet, /testdata/tpch/plan_sf0.02/supplier/2.parquet], [/testdata/tpch/plan_sf0.02/supplier/3.parquet, /testdata/tpch/plan_sf0.02/supplier/4.parquet, /testdata/tpch/plan_sf0.02/supplier/5.parquet], [/testdata/tpch/plan_sf0.02/supplier/6.parquet, /testdata/tpch/plan_sf0.02/supplier/7.parquet, /testdata/tpch/plan_sf0.02/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] - │ 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 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([l_suppkey@2, l_partkey@1], 24), input_partitions=6 + │ ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_partkey@2 as l_partkey, l_suppkey@3 as l_suppkey, l_quantity@4 as l_quantity, l_extendedprice@5 as l_extendedprice, l_discount@6 as l_discount, s_nationkey@0 as s_nationkey] + │ CoalesceBatchesExec: target_batch_size=8192 + │ 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={6 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet, /testdata/tpch/plan_sf0.02/supplier/10.parquet, /testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet, /testdata/tpch/plan_sf0.02/supplier/13.parquet, /testdata/tpch/plan_sf0.02/supplier/14.parquet], [/testdata/tpch/plan_sf0.02/supplier/15.parquet, /testdata/tpch/plan_sf0.02/supplier/16.parquet, /testdata/tpch/plan_sf0.02/supplier/2.parquet], [/testdata/tpch/plan_sf0.02/supplier/3.parquet, /testdata/tpch/plan_sf0.02/supplier/4.parquet, /testdata/tpch/plan_sf0.02/supplier/5.parquet], [/testdata/tpch/plan_sf0.02/supplier/6.parquet, /testdata/tpch/plan_sf0.02/supplier/7.parquet, /testdata/tpch/plan_sf0.02/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + │ 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 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 @@ -495,9 +494,10 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/part/1.parquet, /testdata/tpch/plan_sf0.02/part/10.parquet, /testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet, /testdata/tpch/plan_sf0.02/part/13.parquet, /testdata/tpch/plan_sf0.02/part/14.parquet], [/testdata/tpch/plan_sf0.02/part/15.parquet, /testdata/tpch/plan_sf0.02/part/16.parquet, /testdata/tpch/plan_sf0.02/part/2.parquet], [/testdata/tpch/plan_sf0.02/part/3.parquet, /testdata/tpch/plan_sf0.02/part/4.parquet, /testdata/tpch/plan_sf0.02/part/5.parquet], [/testdata/tpch/plan_sf0.02/part/6.parquet, /testdata/tpch/plan_sf0.02/part/7.parquet, /testdata/tpch/plan_sf0.02/part/8.parquet], ...]}, projection=[p_partkey, p_name], file_type=parquet, predicate=p_name@1 LIKE %green% └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] t3:[p0..p23] - │ RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 24), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/partsupp/1.parquet, /testdata/tpch/plan_sf0.02/partsupp/10.parquet, /testdata/tpch/plan_sf0.02/partsupp/11.parquet], [/testdata/tpch/plan_sf0.02/partsupp/12.parquet, /testdata/tpch/plan_sf0.02/partsupp/13.parquet, /testdata/tpch/plan_sf0.02/partsupp/14.parquet], [/testdata/tpch/plan_sf0.02/partsupp/15.parquet, /testdata/tpch/plan_sf0.02/partsupp/16.parquet, /testdata/tpch/plan_sf0.02/partsupp/2.parquet], [/testdata/tpch/plan_sf0.02/partsupp/3.parquet, /testdata/tpch/plan_sf0.02/partsupp/4.parquet, /testdata/tpch/plan_sf0.02/partsupp/5.parquet], [/testdata/tpch/plan_sf0.02/partsupp/6.parquet, /testdata/tpch/plan_sf0.02/partsupp/7.parquet, /testdata/tpch/plan_sf0.02/partsupp/8.parquet], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ empty ] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 24), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/partsupp/1.parquet, /testdata/tpch/plan_sf0.02/partsupp/10.parquet, /testdata/tpch/plan_sf0.02/partsupp/11.parquet], [/testdata/tpch/plan_sf0.02/partsupp/12.parquet, /testdata/tpch/plan_sf0.02/partsupp/13.parquet, /testdata/tpch/plan_sf0.02/partsupp/14.parquet], [/testdata/tpch/plan_sf0.02/partsupp/15.parquet, /testdata/tpch/plan_sf0.02/partsupp/16.parquet, /testdata/tpch/plan_sf0.02/partsupp/2.parquet], [/testdata/tpch/plan_sf0.02/partsupp/3.parquet, /testdata/tpch/plan_sf0.02/partsupp/4.parquet, /testdata/tpch/plan_sf0.02/partsupp/5.parquet], [/testdata/tpch/plan_sf0.02/partsupp/6.parquet, /testdata/tpch/plan_sf0.02/partsupp/7.parquet, /testdata/tpch/plan_sf0.02/partsupp/8.parquet], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── "); Ok(()) @@ -515,29 +515,29 @@ mod tests { │ 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 - │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1 + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p17] - │ RepartitionExec: partitioning=Hash([c_custkey@0, c_name@1, c_acctbal@2, c_phone@3, n_name@4, c_address@5, c_comment@6], 18), input_partitions=6 - │ AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@4 as c_acctbal, c_phone@3 as c_phone, n_name@8 as n_name, c_address@2 as c_address, c_comment@5 as c_comment], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - │ ProjectionExec: expr=[c_custkey@1 as c_custkey, c_name@2 as c_name, c_address@3 as c_address, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, l_extendedprice@7 as l_extendedprice, l_discount@8 as l_discount, n_name@0 as n_name] - │ 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={6 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet, /testdata/tpch/plan_sf0.02/nation/10.parquet, /testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet, /testdata/tpch/plan_sf0.02/nation/13.parquet, /testdata/tpch/plan_sf0.02/nation/14.parquet], [/testdata/tpch/plan_sf0.02/nation/15.parquet, /testdata/tpch/plan_sf0.02/nation/16.parquet, /testdata/tpch/plan_sf0.02/nation/2.parquet], [/testdata/tpch/plan_sf0.02/nation/3.parquet, /testdata/tpch/plan_sf0.02/nation/4.parquet, /testdata/tpch/plan_sf0.02/nation/5.parquet], [/testdata/tpch/plan_sf0.02/nation/6.parquet, /testdata/tpch/plan_sf0.02/nation/7.parquet, /testdata/tpch/plan_sf0.02/nation/8.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 - │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/customer/1.parquet, /testdata/tpch/plan_sf0.02/customer/10.parquet, /testdata/tpch/plan_sf0.02/customer/11.parquet], [/testdata/tpch/plan_sf0.02/customer/12.parquet, /testdata/tpch/plan_sf0.02/customer/13.parquet, /testdata/tpch/plan_sf0.02/customer/14.parquet], [/testdata/tpch/plan_sf0.02/customer/15.parquet, /testdata/tpch/plan_sf0.02/customer/16.parquet, /testdata/tpch/plan_sf0.02/customer/2.parquet], [/testdata/tpch/plan_sf0.02/customer/3.parquet, /testdata/tpch/plan_sf0.02/customer/4.parquet, /testdata/tpch/plan_sf0.02/customer/5.parquet], [/testdata/tpch/plan_sf0.02/customer/6.parquet, /testdata/tpch/plan_sf0.02/customer/7.parquet, /testdata/tpch/plan_sf0.02/customer/8.parquet], ...]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], file_type=parquet, predicate=l_returnflag@3 = R AND DynamicFilter [ empty ], pruning_predicate=l_returnflag_null_count@2 != row_count@3 AND l_returnflag_min@0 <= R AND R <= l_returnflag_max@1, required_guarantees=[l_returnflag in (R)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([c_custkey@0, c_name@1, c_acctbal@2, c_phone@3, n_name@4, c_address@5, c_comment@6], 18), input_partitions=6 + │ AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@4 as c_acctbal, c_phone@3 as c_phone, n_name@8 as n_name, c_address@2 as c_address, c_comment@5 as c_comment], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + │ ProjectionExec: expr=[c_custkey@1 as c_custkey, c_name@2 as c_name, c_address@3 as c_address, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, l_extendedprice@7 as l_extendedprice, l_discount@8 as l_discount, n_name@0 as n_name] + │ 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={6 groups: [[/testdata/tpch/plan_sf0.02/nation/1.parquet, /testdata/tpch/plan_sf0.02/nation/10.parquet, /testdata/tpch/plan_sf0.02/nation/11.parquet], [/testdata/tpch/plan_sf0.02/nation/12.parquet, /testdata/tpch/plan_sf0.02/nation/13.parquet, /testdata/tpch/plan_sf0.02/nation/14.parquet], [/testdata/tpch/plan_sf0.02/nation/15.parquet, /testdata/tpch/plan_sf0.02/nation/16.parquet, /testdata/tpch/plan_sf0.02/nation/2.parquet], [/testdata/tpch/plan_sf0.02/nation/3.parquet, /testdata/tpch/plan_sf0.02/nation/4.parquet, /testdata/tpch/plan_sf0.02/nation/5.parquet], [/testdata/tpch/plan_sf0.02/nation/6.parquet, /testdata/tpch/plan_sf0.02/nation/7.parquet, /testdata/tpch/plan_sf0.02/nation/8.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 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/customer/1.parquet, /testdata/tpch/plan_sf0.02/customer/10.parquet, /testdata/tpch/plan_sf0.02/customer/11.parquet], [/testdata/tpch/plan_sf0.02/customer/12.parquet, /testdata/tpch/plan_sf0.02/customer/13.parquet, /testdata/tpch/plan_sf0.02/customer/14.parquet], [/testdata/tpch/plan_sf0.02/customer/15.parquet, /testdata/tpch/plan_sf0.02/customer/16.parquet, /testdata/tpch/plan_sf0.02/customer/2.parquet], [/testdata/tpch/plan_sf0.02/customer/3.parquet, /testdata/tpch/plan_sf0.02/customer/4.parquet, /testdata/tpch/plan_sf0.02/customer/5.parquet], [/testdata/tpch/plan_sf0.02/customer/6.parquet, /testdata/tpch/plan_sf0.02/customer/7.parquet, /testdata/tpch/plan_sf0.02/customer/8.parquet], ...]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], file_type=parquet, predicate=l_returnflag@3 = R AND DynamicFilter [ empty ], pruning_predicate=l_returnflag_null_count@2 != row_count@3 AND l_returnflag_min@0 <= R AND R <= l_returnflag_max@1, required_guarantees=[l_returnflag in (R)] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 @@ -616,30 +616,30 @@ mod tests { │ 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 - │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] - │ RepartitionExec: partitioning=Hash([l_shipmode@0], 18), input_partitions=6 - │ AggregateExec: mode=Partial, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] - │ CoalesceBatchesExec: target_batch_size=8192 - │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] - │ CoalesceBatchesExec: target_batch_size=8192 + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([l_shipmode@0], 18), input_partitions=6 + │ AggregateExec: mode=Partial, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4 - │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] - │ RepartitionExec: partitioning=Hash([l_orderkey@0], 18), input_partitions=2 - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, projection=[l_orderkey@0, l_shipmode@4] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_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)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 18), input_partitions=2 + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, projection=[l_orderkey@0, l_shipmode@4] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], file_type=parquet, predicate=(l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, pruning_predicate=(l_shipmode_null_count@2 != row_count@3 AND l_shipmode_min@0 <= MAIL AND MAIL <= l_shipmode_max@1 OR l_shipmode_null_count@2 != row_count@3 AND l_shipmode_min@0 <= SHIP AND SHIP <= l_shipmode_max@1) AND l_receiptdate_null_count@5 != row_count@3 AND l_receiptdate_max@4 >= 1994-01-01 AND l_receiptdate_null_count@5 != row_count@3 AND l_receiptdate_min@6 < 1995-01-01, required_guarantees=[l_shipmode in (MAIL, SHIP)] └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] - │ RepartitionExec: partitioning=Hash([o_orderkey@0], 18), input_partitions=2 - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet, /testdata/tpch/plan_sf0.02/orders/10.parquet, /testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet, /testdata/tpch/plan_sf0.02/orders/13.parquet, /testdata/tpch/plan_sf0.02/orders/14.parquet], [/testdata/tpch/plan_sf0.02/orders/15.parquet, /testdata/tpch/plan_sf0.02/orders/16.parquet, /testdata/tpch/plan_sf0.02/orders/2.parquet], [/testdata/tpch/plan_sf0.02/orders/3.parquet, /testdata/tpch/plan_sf0.02/orders/4.parquet, /testdata/tpch/plan_sf0.02/orders/5.parquet], [/testdata/tpch/plan_sf0.02/orders/6.parquet, /testdata/tpch/plan_sf0.02/orders/7.parquet, /testdata/tpch/plan_sf0.02/orders/8.parquet], ...]}, projection=[o_orderkey, o_orderpriority], file_type=parquet, predicate=DynamicFilter [ empty ] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 18), input_partitions=2 + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet, /testdata/tpch/plan_sf0.02/orders/10.parquet, /testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet, /testdata/tpch/plan_sf0.02/orders/13.parquet, /testdata/tpch/plan_sf0.02/orders/14.parquet], [/testdata/tpch/plan_sf0.02/orders/15.parquet, /testdata/tpch/plan_sf0.02/orders/16.parquet, /testdata/tpch/plan_sf0.02/orders/2.parquet], [/testdata/tpch/plan_sf0.02/orders/3.parquet, /testdata/tpch/plan_sf0.02/orders/4.parquet, /testdata/tpch/plan_sf0.02/orders/5.parquet], [/testdata/tpch/plan_sf0.02/orders/6.parquet, /testdata/tpch/plan_sf0.02/orders/7.parquet, /testdata/tpch/plan_sf0.02/orders/8.parquet], ...]}, projection=[o_orderkey, o_orderpriority], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── "#); Ok(()) @@ -657,26 +657,26 @@ mod tests { │ 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 - │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] - │ RepartitionExec: partitioning=Hash([c_count@0], 12), input_partitions=6 - │ AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(Int64(1))] - │ ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count] - │ AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] - │ CoalesceBatchesExec: target_batch_size=8192 + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([c_count@0], 12), input_partitions=6 + │ AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(Int64(1))] + │ ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count] + │ AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p17] - │ RepartitionExec: partitioning=Hash([c_custkey@0], 18), input_partitions=6 - │ AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] - │ ProjectionExec: expr=[c_custkey@1 as c_custkey, o_orderkey@0 as o_orderkey] - │ CoalesceBatchesExec: target_batch_size=8192 - │ HashJoinExec: mode=CollectLeft, join_type=Right, on=[(o_custkey@1, c_custkey@0)], projection=[o_orderkey@0, c_custkey@2] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/customer/1.parquet, /testdata/tpch/plan_sf0.02/customer/10.parquet, /testdata/tpch/plan_sf0.02/customer/11.parquet], [/testdata/tpch/plan_sf0.02/customer/12.parquet, /testdata/tpch/plan_sf0.02/customer/13.parquet, /testdata/tpch/plan_sf0.02/customer/14.parquet], [/testdata/tpch/plan_sf0.02/customer/15.parquet, /testdata/tpch/plan_sf0.02/customer/16.parquet, /testdata/tpch/plan_sf0.02/customer/2.parquet], [/testdata/tpch/plan_sf0.02/customer/3.parquet, /testdata/tpch/plan_sf0.02/customer/4.parquet, /testdata/tpch/plan_sf0.02/customer/5.parquet], [/testdata/tpch/plan_sf0.02/customer/6.parquet, /testdata/tpch/plan_sf0.02/customer/7.parquet, /testdata/tpch/plan_sf0.02/customer/8.parquet], ...]}, projection=[c_custkey], file_type=parquet + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([c_custkey@0], 18), input_partitions=6 + │ AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] + │ ProjectionExec: expr=[c_custkey@1 as c_custkey, o_orderkey@0 as o_orderkey] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Right, on=[(o_custkey@1, c_custkey@0)], projection=[o_orderkey@0, c_custkey@2] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/customer/1.parquet, /testdata/tpch/plan_sf0.02/customer/10.parquet, /testdata/tpch/plan_sf0.02/customer/11.parquet], [/testdata/tpch/plan_sf0.02/customer/12.parquet, /testdata/tpch/plan_sf0.02/customer/13.parquet, /testdata/tpch/plan_sf0.02/customer/14.parquet], [/testdata/tpch/plan_sf0.02/customer/15.parquet, /testdata/tpch/plan_sf0.02/customer/16.parquet, /testdata/tpch/plan_sf0.02/customer/2.parquet], [/testdata/tpch/plan_sf0.02/customer/3.parquet, /testdata/tpch/plan_sf0.02/customer/4.parquet, /testdata/tpch/plan_sf0.02/customer/5.parquet], [/testdata/tpch/plan_sf0.02/customer/6.parquet, /testdata/tpch/plan_sf0.02/customer/7.parquet, /testdata/tpch/plan_sf0.02/customer/8.parquet], ...]}, projection=[c_custkey], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 @@ -726,31 +726,31 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet, /testdata/tpch/plan_sf0.02/supplier/10.parquet, /testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet, /testdata/tpch/plan_sf0.02/supplier/13.parquet, /testdata/tpch/plan_sf0.02/supplier/14.parquet], [/testdata/tpch/plan_sf0.02/supplier/15.parquet, /testdata/tpch/plan_sf0.02/supplier/16.parquet, /testdata/tpch/plan_sf0.02/supplier/2.parquet], [/testdata/tpch/plan_sf0.02/supplier/3.parquet, /testdata/tpch/plan_sf0.02/supplier/4.parquet, /testdata/tpch/plan_sf0.02/supplier/5.parquet], [/testdata/tpch/plan_sf0.02/supplier/6.parquet, /testdata/tpch/plan_sf0.02/supplier/7.parquet, /testdata/tpch/plan_sf0.02/supplier/8.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 - │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[max(revenue0.total_revenue)] │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] │ AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - │ CoalesceBatchesExec: target_batch_size=8192 - │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] t3:[p0..p11] - │ RepartitionExec: partitioning=Hash([l_suppkey@0], 12), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_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=[] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([l_suppkey@0], 12), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1996-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1996-04-01, required_guarantees=[] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ RepartitionExec: partitioning=Hash([l_suppkey@0], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_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=[] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([l_suppkey@0], 6), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_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(()) @@ -768,29 +768,29 @@ mod tests { │ 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 - │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=3 + │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── ┌───── Stage 4 ── Tasks: t0:[p0..p11] t1:[p0..p11] t2:[p0..p11] - │ RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2], 12), input_partitions=6 - │ 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 + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2], 12), input_partitions=6 + │ AggregateExec: mode=Partial, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[count(alias1)] + │ AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, alias1@3 as alias1], aggr=[] │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p17] - │ RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 18), input_partitions=6 - │ AggregateExec: mode=Partial, gby=[p_brand@1 as p_brand, p_type@2 as p_type, p_size@3 as p_size, ps_suppkey@0 as alias1], aggr=[] - │ CoalesceBatchesExec: target_batch_size=8192 - │ HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(s_suppkey@0, ps_suppkey@0)] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 - │ ProjectionExec: expr=[ps_suppkey@3 as ps_suppkey, p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size] - │ CoalesceBatchesExec: target_batch_size=8192 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_brand@1, p_type@2, p_size@3, ps_suppkey@5] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/partsupp/1.parquet, /testdata/tpch/plan_sf0.02/partsupp/10.parquet, /testdata/tpch/plan_sf0.02/partsupp/11.parquet], [/testdata/tpch/plan_sf0.02/partsupp/12.parquet, /testdata/tpch/plan_sf0.02/partsupp/13.parquet, /testdata/tpch/plan_sf0.02/partsupp/14.parquet], [/testdata/tpch/plan_sf0.02/partsupp/15.parquet, /testdata/tpch/plan_sf0.02/partsupp/16.parquet, /testdata/tpch/plan_sf0.02/partsupp/2.parquet], [/testdata/tpch/plan_sf0.02/partsupp/3.parquet, /testdata/tpch/plan_sf0.02/partsupp/4.parquet, /testdata/tpch/plan_sf0.02/partsupp/5.parquet], [/testdata/tpch/plan_sf0.02/partsupp/6.parquet, /testdata/tpch/plan_sf0.02/partsupp/7.parquet, /testdata/tpch/plan_sf0.02/partsupp/8.parquet], ...]}, projection=[ps_partkey, ps_suppkey], file_type=parquet, predicate=DynamicFilter [ empty ] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 18), input_partitions=6 + │ AggregateExec: mode=Partial, gby=[p_brand@1 as p_brand, p_type@2 as p_type, p_size@3 as p_size, ps_suppkey@0 as alias1], aggr=[] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(s_suppkey@0, ps_suppkey@0)] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[ps_suppkey@3 as ps_suppkey, p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_brand@1, p_type@2, p_size@3, ps_suppkey@5] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/partsupp/1.parquet, /testdata/tpch/plan_sf0.02/partsupp/10.parquet, /testdata/tpch/plan_sf0.02/partsupp/11.parquet], [/testdata/tpch/plan_sf0.02/partsupp/12.parquet, /testdata/tpch/plan_sf0.02/partsupp/13.parquet, /testdata/tpch/plan_sf0.02/partsupp/14.parquet], [/testdata/tpch/plan_sf0.02/partsupp/15.parquet, /testdata/tpch/plan_sf0.02/partsupp/16.parquet, /testdata/tpch/plan_sf0.02/partsupp/2.parquet], [/testdata/tpch/plan_sf0.02/partsupp/3.parquet, /testdata/tpch/plan_sf0.02/partsupp/4.parquet, /testdata/tpch/plan_sf0.02/partsupp/5.parquet], [/testdata/tpch/plan_sf0.02/partsupp/6.parquet, /testdata/tpch/plan_sf0.02/partsupp/7.parquet, /testdata/tpch/plan_sf0.02/partsupp/8.parquet], ...]}, projection=[ps_partkey, ps_suppkey], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 @@ -828,8 +828,7 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_partkey, l_quantity, l_extendedprice], file_type=parquet, predicate=DynamicFilter [ empty ] │ 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 - │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 @@ -838,10 +837,11 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/part/1.parquet, /testdata/tpch/plan_sf0.02/part/10.parquet, /testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet, /testdata/tpch/plan_sf0.02/part/13.parquet, /testdata/tpch/plan_sf0.02/part/14.parquet], [/testdata/tpch/plan_sf0.02/part/15.parquet, /testdata/tpch/plan_sf0.02/part/16.parquet, /testdata/tpch/plan_sf0.02/part/2.parquet], [/testdata/tpch/plan_sf0.02/part/3.parquet, /testdata/tpch/plan_sf0.02/part/4.parquet, /testdata/tpch/plan_sf0.02/part/5.parquet], [/testdata/tpch/plan_sf0.02/part/6.parquet, /testdata/tpch/plan_sf0.02/part/7.parquet, /testdata/tpch/plan_sf0.02/part/8.parquet], ...]}, projection=[p_partkey, p_brand, p_container], file_type=parquet, predicate=p_brand@1 = Brand#23 AND p_container@2 = MED BOX, pruning_predicate=p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#23 AND Brand#23 <= p_brand_max@1 AND p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED BOX AND MED BOX <= p_container_max@5, required_guarantees=[p_brand in (Brand#23), p_container in (MED BOX)] └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ RepartitionExec: partitioning=Hash([l_partkey@0], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_partkey, l_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([l_partkey@0], 6), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_partkey, l_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── "); Ok(()) @@ -858,38 +858,38 @@ mod tests { ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ SortExec: expr=[o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST], preserve_partitioning=[true] │ AggregateExec: mode=FinalPartitioned, gby=[c_name@0 as c_name, c_custkey@1 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@3 as o_orderdate, o_totalprice@4 as o_totalprice], aggr=[sum(lineitem.l_quantity)] - │ CoalesceBatchesExec: target_batch_size=8192 - │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p17] - │ RepartitionExec: partitioning=Hash([c_name@0, c_custkey@1, o_orderkey@2, o_orderdate@3, o_totalprice@4], 18), input_partitions=6 - │ AggregateExec: mode=Partial, gby=[c_name@1 as c_name, c_custkey@0 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@4 as o_orderdate, o_totalprice@3 as o_totalprice], aggr=[sum(lineitem.l_quantity)] - │ CoalesceBatchesExec: target_batch_size=8192 - │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(l_orderkey@0, o_orderkey@2)] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=18, input_tasks=3 - │ 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={6 groups: [[/testdata/tpch/plan_sf0.02/customer/1.parquet, /testdata/tpch/plan_sf0.02/customer/10.parquet, /testdata/tpch/plan_sf0.02/customer/11.parquet], [/testdata/tpch/plan_sf0.02/customer/12.parquet, /testdata/tpch/plan_sf0.02/customer/13.parquet, /testdata/tpch/plan_sf0.02/customer/14.parquet], [/testdata/tpch/plan_sf0.02/customer/15.parquet, /testdata/tpch/plan_sf0.02/customer/16.parquet, /testdata/tpch/plan_sf0.02/customer/2.parquet], [/testdata/tpch/plan_sf0.02/customer/3.parquet, /testdata/tpch/plan_sf0.02/customer/4.parquet, /testdata/tpch/plan_sf0.02/customer/5.parquet], [/testdata/tpch/plan_sf0.02/customer/6.parquet, /testdata/tpch/plan_sf0.02/customer/7.parquet, /testdata/tpch/plan_sf0.02/customer/8.parquet], ...]}, projection=[c_custkey, c_name], file_type=parquet - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet, /testdata/tpch/plan_sf0.02/orders/10.parquet, /testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet, /testdata/tpch/plan_sf0.02/orders/13.parquet, /testdata/tpch/plan_sf0.02/orders/14.parquet], [/testdata/tpch/plan_sf0.02/orders/15.parquet, /testdata/tpch/plan_sf0.02/orders/16.parquet, /testdata/tpch/plan_sf0.02/orders/2.parquet], [/testdata/tpch/plan_sf0.02/orders/3.parquet, /testdata/tpch/plan_sf0.02/orders/4.parquet, /testdata/tpch/plan_sf0.02/orders/5.parquet], [/testdata/tpch/plan_sf0.02/orders/6.parquet, /testdata/tpch/plan_sf0.02/orders/7.parquet, /testdata/tpch/plan_sf0.02/orders/8.parquet], ...]}, projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate], file_type=parquet, predicate=DynamicFilter [ empty ] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([c_name@0, c_custkey@1, o_orderkey@2, o_orderdate@3, o_totalprice@4], 18), input_partitions=6 + │ AggregateExec: mode=Partial, gby=[c_name@1 as c_name, c_custkey@0 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@4 as o_orderdate, o_totalprice@3 as o_totalprice], aggr=[sum(lineitem.l_quantity)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(l_orderkey@0, o_orderkey@2)] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkCoalesceExec: output_partitions=18, input_tasks=3 + │ 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={6 groups: [[/testdata/tpch/plan_sf0.02/customer/1.parquet, /testdata/tpch/plan_sf0.02/customer/10.parquet, /testdata/tpch/plan_sf0.02/customer/11.parquet], [/testdata/tpch/plan_sf0.02/customer/12.parquet, /testdata/tpch/plan_sf0.02/customer/13.parquet, /testdata/tpch/plan_sf0.02/customer/14.parquet], [/testdata/tpch/plan_sf0.02/customer/15.parquet, /testdata/tpch/plan_sf0.02/customer/16.parquet, /testdata/tpch/plan_sf0.02/customer/2.parquet], [/testdata/tpch/plan_sf0.02/customer/3.parquet, /testdata/tpch/plan_sf0.02/customer/4.parquet, /testdata/tpch/plan_sf0.02/customer/5.parquet], [/testdata/tpch/plan_sf0.02/customer/6.parquet, /testdata/tpch/plan_sf0.02/customer/7.parquet, /testdata/tpch/plan_sf0.02/customer/8.parquet], ...]}, projection=[c_custkey, c_name], file_type=parquet + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet, /testdata/tpch/plan_sf0.02/orders/10.parquet, /testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet, /testdata/tpch/plan_sf0.02/orders/13.parquet, /testdata/tpch/plan_sf0.02/orders/14.parquet], [/testdata/tpch/plan_sf0.02/orders/15.parquet, /testdata/tpch/plan_sf0.02/orders/16.parquet, /testdata/tpch/plan_sf0.02/orders/2.parquet], [/testdata/tpch/plan_sf0.02/orders/3.parquet, /testdata/tpch/plan_sf0.02/orders/4.parquet, /testdata/tpch/plan_sf0.02/orders/5.parquet], [/testdata/tpch/plan_sf0.02/orders/6.parquet, /testdata/tpch/plan_sf0.02/orders/7.parquet, /testdata/tpch/plan_sf0.02/orders/8.parquet], ...]}, projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate], file_type=parquet, predicate=DynamicFilter [ empty ] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_quantity], file_type=parquet, predicate=DynamicFilter [ empty ] └────────────────────────────────────────────────── ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0] │ AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] - │ CoalesceBatchesExec: target_batch_size=8192 - │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] t3:[p0..p17] - │ RepartitionExec: partitioning=Hash([l_orderkey@0], 18), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_quantity], file_type=parquet + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 18), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_quantity], file_type=parquet └────────────────────────────────────────────────── "); Ok(()) @@ -947,8 +947,7 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/partsupp/1.parquet, /testdata/tpch/plan_sf0.02/partsupp/10.parquet, /testdata/tpch/plan_sf0.02/partsupp/11.parquet], [/testdata/tpch/plan_sf0.02/partsupp/12.parquet, /testdata/tpch/plan_sf0.02/partsupp/13.parquet, /testdata/tpch/plan_sf0.02/partsupp/14.parquet], [/testdata/tpch/plan_sf0.02/partsupp/15.parquet, /testdata/tpch/plan_sf0.02/partsupp/16.parquet, /testdata/tpch/plan_sf0.02/partsupp/2.parquet], [/testdata/tpch/plan_sf0.02/partsupp/3.parquet, /testdata/tpch/plan_sf0.02/partsupp/4.parquet, /testdata/tpch/plan_sf0.02/partsupp/5.parquet], [/testdata/tpch/plan_sf0.02/partsupp/6.parquet, /testdata/tpch/plan_sf0.02/partsupp/7.parquet, /testdata/tpch/plan_sf0.02/partsupp/8.parquet], ...]}, projection=[ps_partkey, ps_suppkey, ps_availqty], file_type=parquet │ ProjectionExec: expr=[0.5 * CAST(sum(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * sum(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey] │ AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] - │ CoalesceBatchesExec: target_batch_size=8192 - │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=4 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=4 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 @@ -963,12 +962,13 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/part/1.parquet, /testdata/tpch/plan_sf0.02/part/10.parquet, /testdata/tpch/plan_sf0.02/part/11.parquet], [/testdata/tpch/plan_sf0.02/part/12.parquet, /testdata/tpch/plan_sf0.02/part/13.parquet, /testdata/tpch/plan_sf0.02/part/14.parquet], [/testdata/tpch/plan_sf0.02/part/15.parquet, /testdata/tpch/plan_sf0.02/part/16.parquet, /testdata/tpch/plan_sf0.02/part/2.parquet], [/testdata/tpch/plan_sf0.02/part/3.parquet, /testdata/tpch/plan_sf0.02/part/4.parquet, /testdata/tpch/plan_sf0.02/part/5.parquet], [/testdata/tpch/plan_sf0.02/part/6.parquet, /testdata/tpch/plan_sf0.02/part/7.parquet, /testdata/tpch/plan_sf0.02/part/8.parquet], ...]}, projection=[p_partkey, p_name], file_type=parquet, predicate=p_name@1 LIKE forest%, pruning_predicate=p_name_null_count@2 != row_count@3 AND p_name_min@0 <= foresu AND forest <= p_name_max@1, required_guarantees=[] └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] - │ RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 6), input_partitions=2 - │ AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2] - │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND DynamicFilter [ empty ], 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=[] + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 6), input_partitions=2 + │ AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND DynamicFilter [ empty ], 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(()) @@ -986,37 +986,37 @@ mod tests { │ 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 - │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p17] - │ RepartitionExec: partitioning=Hash([s_name@0], 18), input_partitions=6 - │ 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 + │ RepartitionExec: partitioning=Hash([s_name@0], 18), input_partitions=6 + │ 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 + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@2)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] + │ CoalescePartitionsExec + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + │ CoalesceBatchesExec: target_batch_size=8192 + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] + │ CoalescePartitionsExec + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet, /testdata/tpch/plan_sf0.02/supplier/10.parquet, /testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet, /testdata/tpch/plan_sf0.02/supplier/13.parquet, /testdata/tpch/plan_sf0.02/supplier/14.parquet], [/testdata/tpch/plan_sf0.02/supplier/15.parquet, /testdata/tpch/plan_sf0.02/supplier/16.parquet, /testdata/tpch/plan_sf0.02/supplier/2.parquet], [/testdata/tpch/plan_sf0.02/supplier/3.parquet, /testdata/tpch/plan_sf0.02/supplier/4.parquet, /testdata/tpch/plan_sf0.02/supplier/5.parquet], [/testdata/tpch/plan_sf0.02/supplier/6.parquet, /testdata/tpch/plan_sf0.02/supplier/7.parquet, /testdata/tpch/plan_sf0.02/supplier/8.parquet], ...]}, projection=[s_suppkey, s_name, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] + │ CoalesceBatchesExec: target_batch_size=8192 + │ FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@3 > l_commitdate@2 AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_suppkey], file_type=parquet │ 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 - │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 - │ CoalesceBatchesExec: target_batch_size=8192 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@2)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] - │ CoalescePartitionsExec - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 - │ CoalesceBatchesExec: target_batch_size=8192 - │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] - │ CoalescePartitionsExec - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/supplier/1.parquet, /testdata/tpch/plan_sf0.02/supplier/10.parquet, /testdata/tpch/plan_sf0.02/supplier/11.parquet], [/testdata/tpch/plan_sf0.02/supplier/12.parquet, /testdata/tpch/plan_sf0.02/supplier/13.parquet, /testdata/tpch/plan_sf0.02/supplier/14.parquet], [/testdata/tpch/plan_sf0.02/supplier/15.parquet, /testdata/tpch/plan_sf0.02/supplier/16.parquet, /testdata/tpch/plan_sf0.02/supplier/2.parquet], [/testdata/tpch/plan_sf0.02/supplier/3.parquet, /testdata/tpch/plan_sf0.02/supplier/4.parquet, /testdata/tpch/plan_sf0.02/supplier/5.parquet], [/testdata/tpch/plan_sf0.02/supplier/6.parquet, /testdata/tpch/plan_sf0.02/supplier/7.parquet, /testdata/tpch/plan_sf0.02/supplier/8.parquet], ...]}, projection=[s_suppkey, s_name, s_nationkey], file_type=parquet, predicate=DynamicFilter [ empty ] - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@3 > l_commitdate@2 AND DynamicFilter [ empty ] AND DynamicFilter [ empty ] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_suppkey], file_type=parquet - │ CoalesceBatchesExec: target_batch_size=8192 - │ FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@3 > l_commitdate@2 + │ FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/lineitem/1.parquet, /testdata/tpch/plan_sf0.02/lineitem/10.parquet, /testdata/tpch/plan_sf0.02/lineitem/11.parquet], [/testdata/tpch/plan_sf0.02/lineitem/12.parquet, /testdata/tpch/plan_sf0.02/lineitem/13.parquet, /testdata/tpch/plan_sf0.02/lineitem/14.parquet], [/testdata/tpch/plan_sf0.02/lineitem/15.parquet, /testdata/tpch/plan_sf0.02/lineitem/16.parquet, /testdata/tpch/plan_sf0.02/lineitem/2.parquet], [/testdata/tpch/plan_sf0.02/lineitem/3.parquet, /testdata/tpch/plan_sf0.02/lineitem/4.parquet, /testdata/tpch/plan_sf0.02/lineitem/5.parquet], [/testdata/tpch/plan_sf0.02/lineitem/6.parquet, /testdata/tpch/plan_sf0.02/lineitem/7.parquet, /testdata/tpch/plan_sf0.02/lineitem/8.parquet], ...]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@3 > l_commitdate@2 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 @@ -1046,23 +1046,23 @@ mod tests { │ 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 - │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1 + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── ┌───── Stage 3 ── Tasks: t0:[p0..p17] - │ RepartitionExec: partitioning=Hash([cntrycode@0], 18), input_partitions=6 - │ AggregateExec: mode=Partial, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)] - │ ProjectionExec: expr=[substr(c_phone@1, 1, 2) as cntrycode, c_acctbal@2 as c_acctbal] - │ NestedLoopJoinExec: join_type=Inner, filter=join_proj_push_down_1@1 > avg(customer.c_acctbal)@0, projection=[avg(customer.c_acctbal)@0, c_phone@1, c_acctbal@2] - │ AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)] - │ CoalescePartitionsExec - │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 - │ ProjectionExec: expr=[c_phone@0 as c_phone, c_acctbal@1 as c_acctbal, CAST(c_acctbal@1 AS Decimal128(19, 6)) as join_proj_push_down_1] - │ 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 - │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 - │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet, /testdata/tpch/plan_sf0.02/orders/10.parquet, /testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet, /testdata/tpch/plan_sf0.02/orders/13.parquet, /testdata/tpch/plan_sf0.02/orders/14.parquet], [/testdata/tpch/plan_sf0.02/orders/15.parquet, /testdata/tpch/plan_sf0.02/orders/16.parquet, /testdata/tpch/plan_sf0.02/orders/2.parquet], [/testdata/tpch/plan_sf0.02/orders/3.parquet, /testdata/tpch/plan_sf0.02/orders/4.parquet, /testdata/tpch/plan_sf0.02/orders/5.parquet], [/testdata/tpch/plan_sf0.02/orders/6.parquet, /testdata/tpch/plan_sf0.02/orders/7.parquet, /testdata/tpch/plan_sf0.02/orders/8.parquet], ...]}, projection=[o_custkey], file_type=parquet + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([cntrycode@0], 18), input_partitions=6 + │ AggregateExec: mode=Partial, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)] + │ ProjectionExec: expr=[substr(c_phone@1, 1, 2) as cntrycode, c_acctbal@2 as c_acctbal] + │ NestedLoopJoinExec: join_type=Inner, filter=join_proj_push_down_1@1 > avg(customer.c_acctbal)@0, projection=[avg(customer.c_acctbal)@0, c_phone@1, c_acctbal@2] + │ AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)] + │ CoalescePartitionsExec + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[c_phone@0 as c_phone, c_acctbal@1 as c_acctbal, CAST(c_acctbal@1 AS Decimal128(19, 6)) as join_proj_push_down_1] + │ 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 + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/plan_sf0.02/orders/1.parquet, /testdata/tpch/plan_sf0.02/orders/10.parquet, /testdata/tpch/plan_sf0.02/orders/11.parquet], [/testdata/tpch/plan_sf0.02/orders/12.parquet, /testdata/tpch/plan_sf0.02/orders/13.parquet, /testdata/tpch/plan_sf0.02/orders/14.parquet], [/testdata/tpch/plan_sf0.02/orders/15.parquet, /testdata/tpch/plan_sf0.02/orders/16.parquet, /testdata/tpch/plan_sf0.02/orders/2.parquet], [/testdata/tpch/plan_sf0.02/orders/3.parquet, /testdata/tpch/plan_sf0.02/orders/4.parquet, /testdata/tpch/plan_sf0.02/orders/5.parquet], [/testdata/tpch/plan_sf0.02/orders/6.parquet, /testdata/tpch/plan_sf0.02/orders/7.parquet, /testdata/tpch/plan_sf0.02/orders/8.parquet], ...]}, projection=[o_custkey], file_type=parquet └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] diff --git a/tests/udfs.rs b/tests/udfs.rs index 0a758496..aea172fc 100644 --- a/tests/udfs.rs +++ b/tests/udfs.rs @@ -71,8 +71,9 @@ mod tests { │ [Stage 2] => NetworkShuffleExec: output_partitions=1, input_tasks=2 └────────────────────────────────────────────────── ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p0..p1] - │ RepartitionExec: partitioning=Hash([test_udf(1)], 2), input_partitions=1 - │ EmptyExec + │ CoalesceBatchesExec: target_batch_size=8192 + │ RepartitionExec: partitioning=Hash([test_udf(1)], 2), input_partitions=1 + │ EmptyExec └────────────────────────────────────────────────── ", );