From 700e5c2a2cce0142a689e4b4d687b531bf73e86b Mon Sep 17 00:00:00 2001 From: Jayant Shrivastava Date: Fri, 3 Oct 2025 11:17:50 -0400 Subject: [PATCH] implement distributed EXPLAIN ANALYZE This change adds support for displaying a distributed EXPLAIN ANALYZE output. It updates the TPCH validation tests to assert the EXPLAIN ANALYZE output for each query. Implemenation notes: - Adds `src/explain.rs` to store helpers for EXPLAIN - Adds a `Option` field to `StageExec` to contain extra information for display purposes. - Given that the metrics are collected by the root stage, the first step is to rewrite all the `StageExec` nodes in a plan to have this context - Then, at display-time, when displaying a task, we re-write each task plan to use the metrics from the `DislplayCtx` Informs: #123 Remaning work: - disable any metrics propagation if not running EXPLAIN ANALYZE as it adds extra overhead - graphviz support - add docs + exalidraw --- .gitignore | 3 +- src/execution_plans/metrics.rs | 5 + src/execution_plans/mod.rs | 2 +- src/execution_plans/stage.rs | 255 +++- src/explain.rs | 85 ++ src/flight_service/do_get.rs | 1 + src/lib.rs | 2 + src/metrics/mod.rs | 3 +- src/protobuf/stage_proto.rs | 2 + src/test_utils/insta.rs | 128 ++ tests/tpch_validation_test.rs | 2351 +++++++++++++++++++++++++++++++- 11 files changed, 2774 insertions(+), 63 deletions(-) create mode 100644 src/explain.rs diff --git a/.gitignore b/.gitignore index be0175d..d6e1072 100644 --- a/.gitignore +++ b/.gitignore @@ -1,4 +1,5 @@ /.idea /target /benchmarks/data/ -testdata/tpch/data/ \ No newline at end of file +testdata/tpch/data/ +*/**/*.pending-snap \ No newline at end of file diff --git a/src/execution_plans/metrics.rs b/src/execution_plans/metrics.rs index b424c67..1ed51a3 100644 --- a/src/execution_plans/metrics.rs +++ b/src/execution_plans/metrics.rs @@ -27,6 +27,11 @@ impl MetricsWrapperExec { children: None, } } + + /// Returns the inner execution plan. + pub(crate) fn get_inner(&self) -> &Arc { + &self.inner + } } /// MetricsWrapperExec is invisible during display. diff --git a/src/execution_plans/mod.rs b/src/execution_plans/mod.rs index bf421e6..e966813 100644 --- a/src/execution_plans/mod.rs +++ b/src/execution_plans/mod.rs @@ -10,4 +10,4 @@ pub use network_shuffle::{NetworkShuffleExec, NetworkShuffleReadyExec}; pub use partition_isolator::PartitionIsolatorExec; pub(crate) use stage::InputStage; pub use stage::display_plan_graphviz; -pub use stage::{DistributedTaskContext, ExecutionTask, StageExec}; +pub use stage::{DisplayCtx, DistributedTaskContext, ExecutionTask, StageExec}; diff --git a/src/execution_plans/stage.rs b/src/execution_plans/stage.rs index effd075..7a6b4f9 100644 --- a/src/execution_plans/stage.rs +++ b/src/execution_plans/stage.rs @@ -1,9 +1,12 @@ use crate::channel_resolver_ext::get_distributed_channel_resolver; +use crate::execution_plans::MetricsWrapperExec; use crate::execution_plans::NetworkCoalesceExec; +use crate::metrics::TaskMetricsRewriter; use crate::{ChannelResolver, NetworkShuffleExec, PartitionIsolatorExec}; use datafusion::common::{exec_err, internal_datafusion_err, internal_err, plan_err}; use datafusion::error::{DataFusionError, Result}; use datafusion::execution::TaskContext; +use datafusion::physical_plan::display::DisplayableExecutionPlan; use datafusion::physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, displayable, }; @@ -91,6 +94,8 @@ pub struct StageExec { pub tasks: Vec, /// tree depth of our location in the stage tree, used for display only pub depth: usize, + /// Optional extra information used at display time. + pub display_ctx: Option, } /// A [StageExec] that is the input of another [StageExec]. @@ -192,6 +197,7 @@ impl StageExec { .collect(), tasks: vec![ExecutionTask { url: None }; n_tasks], depth: 0, + display_ctx: None, } } @@ -239,6 +245,7 @@ impl StageExec { inputs: assigned_input_stages, tasks: assigned_tasks, depth: self.depth, + display_ctx: self.display_ctx.clone(), }; Ok(assigned_stage) @@ -292,9 +299,31 @@ impl ExecutionPlan for StageExec { fn with_new_children( self: Arc, - _children: Vec>, + children: Vec>, ) -> Result> { - plan_err!("with_new_children() not supported for StageExec") + let num_children = children.len(); + let child_stage_execs = children + .into_iter() + .filter(|child| child.as_any().downcast_ref::().is_some()) + .map(|child| child.as_any().downcast_ref::().unwrap().clone()) + .collect::>(); + if child_stage_execs.len() != num_children { + return plan_err!("The children of a StageExec must all be StageExec"); + } + let stage = StageExec { + query_id: self.query_id, + num: self.num, + name: self.name.clone(), + plan: self.plan.clone(), + inputs: child_stage_execs + .into_iter() + .map(|s| InputStage::Decoded(Arc::new(s))) + .collect(), + tasks: self.tasks.clone(), + depth: self.depth, + display_ctx: self.display_ctx.clone(), + }; + Ok(Arc::new(stage)) } fn properties(&self) -> &datafusion::physical_plan::PlanProperties { @@ -344,9 +373,13 @@ impl ExecutionPlan for StageExec { } } +use crate::metrics::proto::MetricsSetProto; +use crate::protobuf::StageKey; use bytes::Bytes; +use datafusion::common::HashMap; use datafusion::common::tree_node::{TreeNode, TreeNodeRecursion}; use datafusion::physical_expr::Partitioning; + /// Be able to display a nice tree for stages. /// /// The challenge to doing this at the moment is that `TreeRenderVistor` @@ -367,9 +400,44 @@ const LDCORNER: &str = "└"; // Left bottom corner const VERTICAL: &str = "│"; // Vertical line const HORIZONTAL: &str = "─"; // Horizontal line +// Context used to display a StageExec, tasks, and plans. +#[derive(Debug, Clone)] +pub struct DisplayCtx { + metrics: Arc>>, +} + +impl DisplayCtx { + pub fn new(metrics: HashMap>) -> Self { + Self { + metrics: Arc::new(metrics), + } + } +} + +/// TaskFmt is used to control how tasks are displayed when displaying a [StageExec]'s inner plan. +#[derive(Clone, Copy)] +enum TaskFmt { + // Display all tasks + All, + // Display only the task with the given task_id + TaskID { task_id: usize }, +} + impl StageExec { - fn format(&self, plan: &dyn ExecutionPlan, indent: usize, f: &mut String) -> std::fmt::Result { - let mut node_str = displayable(plan).one_line().to_string(); + fn format( + &self, + plan: &dyn ExecutionPlan, + indent: usize, + task_fmt: TaskFmt, + f: &mut String, + ) -> std::fmt::Result { + // If metrics are available, then display the plan with metrics. + let mut node_str = match &self.display_ctx { + None => displayable(plan).one_line().to_string(), + Some(_) => DisplayableExecutionPlan::with_metrics(plan) + .one_line() + .to_string(), + }; node_str.pop(); write!(f, "{} {node_str}", " ".repeat(indent))?; @@ -406,17 +474,29 @@ impl StageExec { )?; } - if let Some(isolator) = plan.as_any().downcast_ref::() { - write!( - f, - " {}", - format_tasks_for_partition_isolator(isolator, &self.tasks) - )?; + let mut maybe_partition_isolator = plan; + // It's possible that the plan node is wrapped in a MetricsWrapperExec for displaying purposes. + // Check if that's the case. + if let Some(wrapper) = plan.as_any().downcast_ref::() { + maybe_partition_isolator = wrapper.get_inner().as_ref(); + } + + if let Some(isolator) = maybe_partition_isolator + .as_any() + .downcast_ref::() + { + let task_info = match task_fmt { + TaskFmt::All => format_tasks_for_partition_isolator(isolator, &self.tasks), + TaskFmt::TaskID { task_id } => { + format_task_for_partition_isolator(isolator, task_id, self.tasks.len()) + } + }; + write!(f, " {}", task_info)?; } writeln!(f)?; for child in plan.children() { - self.format(child.as_ref(), indent + 2, f)?; + self.format(child.as_ref(), indent + 2, task_fmt, f)?; } Ok(()) } @@ -430,32 +510,99 @@ impl DisplayAs for StageExec { write!(f, "{}", self.name) } DisplayFormatType::Verbose => { - writeln!( - f, - "{}{} {} {}", - LTCORNER, - HORIZONTAL.repeat(5), - self.name, - format_tasks_for_stage(self.tasks.len(), &self.plan) - )?; + match &self.display_ctx { + None => { + writeln!( + f, + "{}{} {} {}", + LTCORNER, + HORIZONTAL.repeat(5), + self.name, + format_tasks_for_stage(self.tasks.len(), &self.plan) + )?; - let mut plan_str = String::new(); - self.format(self.plan.as_ref(), 0, &mut plan_str)?; - let plan_str = plan_str - .split('\n') - .filter(|v| !v.is_empty()) - .collect::>() - .join(&format!("\n{}{}", " ".repeat(self.depth), VERTICAL)); - writeln!(f, "{}{}{}", " ".repeat(self.depth), VERTICAL, plan_str)?; - write!( - f, - "{}{}{}", - " ".repeat(self.depth), - LDCORNER, - HORIZONTAL.repeat(50) - )?; + let mut plan_str = String::new(); + self.format(self.plan.as_ref(), 0, TaskFmt::All, &mut plan_str)?; + let plan_str = plan_str + .split('\n') + .filter(|v| !v.is_empty()) + .collect::>() + .join(&format!("\n{}{}", " ".repeat(self.depth), VERTICAL)); + writeln!(f, "{}{}{}", " ".repeat(self.depth), VERTICAL, plan_str)?; + // Add bottom border + write!( + f, + "{}{}{}", + " ".repeat(self.depth), + LDCORNER, + HORIZONTAL.repeat(50) + )?; - Ok(()) + Ok(()) + } + Some(display_ctx) => { + // If metrics are available, always display each task separately. + for (i, _) in self.tasks.iter().enumerate() { + let mut extra_spacing = "".to_string(); + if i > 0 { + // Add newline for each task + writeln!(f)?; + // with_indent() in DisplayableExectutionPlan will not add indentation for tasks, so we add it manually. + extra_spacing = " ".repeat(self.depth); + } + writeln!( + f, + "{}{}{}{} {}", + extra_spacing, + LTCORNER, + HORIZONTAL.repeat(5), + format!(" {} ", self.name), + format_task_for_stage(i, &self.plan), + )?; + // Uniquely identify the task. + let key = StageKey { + query_id: self.query_id.to_string(), + stage_id: self.num as u64, + task_number: i as u64, + }; + + let mut plan_str = String::new(); + let plan = match display_ctx.metrics.get(&key) { + Some(metrics) => { + let result = TaskMetricsRewriter::new(metrics.to_owned()) + .enrich_task_with_metrics(self.plan.clone()); + if let Err(e) = result { + write!(f, "Error enriching task with metrics: {}", e)?; + return Err(std::fmt::Error); + } + result.unwrap() + } + None => self.plan.clone(), + }; + self.format( + plan.as_ref(), + 0, + TaskFmt::TaskID { task_id: i }, + &mut plan_str, + )?; + let plan_str = plan_str + .split('\n') + .filter(|v| !v.is_empty()) + .collect::>() + .join(&format!("\n{}{}", " ".repeat(self.depth), VERTICAL)); + writeln!(f, "{}{}{}", " ".repeat(self.depth), VERTICAL, plan_str)?; + // Add bottom border + write!( + f, + "{}{}{}", + " ".repeat(self.depth), + LDCORNER, + HORIZONTAL.repeat(50) + )?; + } + Ok(()) + } + } } DisplayFormatType::TreeRender => write!( f, @@ -483,6 +630,22 @@ fn format_tasks_for_stage(n_tasks: usize, head: &Arc) -> Stri result } +fn format_task_for_stage(task_number: usize, head: &Arc) -> String { + let partitioning = head.properties().output_partitioning(); + let input_partitions = partitioning.partition_count(); + let hash_shuffle = matches!(partitioning, Partitioning::Hash(_, _)); + let off = task_number * if hash_shuffle { 0 } else { input_partitions }; + + let mut result = "Task ".to_string(); + result += &format!("t{task_number}:["); + result += &(off..(off + input_partitions)) + .map(|v| format!("p{v}")) + .join(","); + result += "] "; + + result +} + fn format_tasks_for_partition_isolator( isolator: &PartitionIsolatorExec, tasks: &[ExecutionTask], @@ -506,6 +669,28 @@ fn format_tasks_for_partition_isolator( result } +fn format_task_for_partition_isolator( + isolator: &PartitionIsolatorExec, + task_number: usize, + num_tasks: usize, +) -> String { + let input_partitions = isolator.input().output_partitioning().partition_count(); + let partition_groups = PartitionIsolatorExec::partition_groups(input_partitions, num_tasks); + + let n: usize = partition_groups.iter().map(|v| v.len()).sum(); + let mut partitions = vec!["__".to_string(); n]; + + let mut result = "Task ".to_string(); + partition_groups + .get(task_number) + .unwrap() + .iter() + .enumerate() + .for_each(|(j, p)| partitions[*p] = format!("p{j}")); + result += &format!("t{task_number}:[{}] ", partitions.join(",")); + result +} + // num_colors must agree with the colorscheme selected from // https://graphviz.org/doc/info/colors.html const NUM_COLORS: usize = 6; diff --git a/src/explain.rs b/src/explain.rs new file mode 100644 index 0000000..9ebac05 --- /dev/null +++ b/src/explain.rs @@ -0,0 +1,85 @@ +use crate::execution_plans::{DisplayCtx, StageExec}; +use crate::metrics::MetricsCollectorResult; +use crate::metrics::TaskMetricsCollector; +use crate::metrics::proto::MetricsSetProto; +use crate::metrics::proto::df_metrics_set_to_proto; +use crate::protobuf::StageKey; +use datafusion::common::tree_node::Transformed; +use datafusion::common::tree_node::TreeNodeRecursion; +use datafusion::common::tree_node::{TreeNode, TreeNodeRewriter}; +use datafusion::error::DataFusionError; +use datafusion::physical_plan::ExecutionPlan; +use datafusion::physical_plan::display::DisplayableExecutionPlan; +use std::sync::Arc; + +/// Rewriter that injects the display context into the [StageExec] nodes in the plan. +pub struct DisplayCtxReWriter { + display_ctx: DisplayCtx, +} + +impl DisplayCtxReWriter { + pub fn new(display_ctx: DisplayCtx) -> Self { + Self { display_ctx } + } + + /// Rewrites the plan to inject the display context into the [StageExec] nodes in the plan. + pub fn rewrite( + mut self, + plan: Arc, + ) -> Result, DataFusionError> { + let transformed = plan.rewrite(&mut self)?; + Ok(transformed.data) + } +} + +impl TreeNodeRewriter for DisplayCtxReWriter { + type Node = Arc; + + fn f_down(&mut self, plan: Self::Node) -> Result, DataFusionError> { + match plan.as_any().downcast_ref::() { + Some(stage_exec) => { + let mut copy = stage_exec.clone(); + copy.display_ctx = Some(self.display_ctx.clone()); + Ok(Transformed::new( + Arc::new(copy), + true, + TreeNodeRecursion::Continue, + )) + } + None => Err(DataFusionError::Internal("expected stage exec".to_string())), + } + } +} + +pub fn explain_analyze(executed: Arc) -> Result { + // Check if the plan is distributed by looking for a root [StageExec]. + let plan = match executed.as_any().downcast_ref::() { + None => executed, + Some(stage_exec) => { + // If the plan was distributed, collect metrics from the coordinating stage exec. + let MetricsCollectorResult { + task_metrics, + mut input_task_metrics, + } = TaskMetricsCollector::new().collect(stage_exec.plan.clone())?; + input_task_metrics.insert( + StageKey { + query_id: stage_exec.query_id.to_string(), + stage_id: stage_exec.num as u64, + task_number: 0, + }, + task_metrics + .into_iter() + .map(|metrics| df_metrics_set_to_proto(&metrics)) + .collect::, DataFusionError>>()?, + ); + + let display_ctx = DisplayCtx::new(input_task_metrics); + // Inject the display context into the plan. + DisplayCtxReWriter::new(display_ctx).rewrite(executed.clone())? + } + }; + + Ok(DisplayableExecutionPlan::new(plan.as_ref()) + .indent(true) + .to_string()) +} diff --git a/src/flight_service/do_get.rs b/src/flight_service/do_get.rs index 03585fc..2bcce75 100644 --- a/src/flight_service/do_get.rs +++ b/src/flight_service/do_get.rs @@ -326,6 +326,7 @@ mod tests { inputs: vec![], tasks, depth: 0, + display_ctx: None, }; let task_keys = [ diff --git a/src/lib.rs b/src/lib.rs index 9958379..7d44e64 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -6,6 +6,7 @@ mod config_extension_ext; mod distributed_ext; mod distributed_physical_optimizer_rule; mod execution_plans; +mod explain; mod flight_service; mod metrics; @@ -21,6 +22,7 @@ pub use execution_plans::{ DistributedTaskContext, ExecutionTask, NetworkCoalesceExec, NetworkShuffleExec, PartitionIsolatorExec, StageExec, }; +pub use explain::explain_analyze; pub use flight_service::{ ArrowFlightEndpoint, DefaultSessionBuilder, DistributedSessionBuilder, DistributedSessionBuilderContext, MappedDistributedSessionBuilder, diff --git a/src/metrics/mod.rs b/src/metrics/mod.rs index b82d8c0..8d6c008 100644 --- a/src/metrics/mod.rs +++ b/src/metrics/mod.rs @@ -3,4 +3,5 @@ pub(crate) mod proto; mod task_metrics_collector; mod task_metrics_rewriter; pub(crate) use metrics_collecting_stream::MetricsCollectingStream; -pub(crate) use task_metrics_collector::TaskMetricsCollector; +pub(crate) use task_metrics_collector::{MetricsCollectorResult, TaskMetricsCollector}; +pub(crate) use task_metrics_rewriter::TaskMetricsRewriter; diff --git a/src/protobuf/stage_proto.rs b/src/protobuf/stage_proto.rs index bcf8b74..26fb6eb 100644 --- a/src/protobuf/stage_proto.rs +++ b/src/protobuf/stage_proto.rs @@ -208,6 +208,7 @@ pub(crate) fn stage_from_proto( inputs, tasks: decode_tasks(msg.tasks)?, depth: 0, + display_ctx: None, }) } @@ -274,6 +275,7 @@ mod tests { inputs: vec![], tasks: vec![], depth: 0, + display_ctx: None, }; // Convert to proto message diff --git a/src/test_utils/insta.rs b/src/test_utils/insta.rs index 54c9225..47f0432 100644 --- a/src/test_utils/insta.rs +++ b/src/test_utils/insta.rs @@ -24,5 +24,133 @@ pub fn settings() -> insta::Settings { ); settings.add_filter(r"\d+\.\.\d+", ".."); + // Metric filters - only replace known metric names to avoid false positives + settings.add_filter(r"output_rows=\d+", "output_rows="); + settings.add_filter( + r"elapsed_compute=[\d.]+[a-zA-Zµnms]+", + "elapsed_compute=", + ); + settings.add_filter(r"spill_count=\d+", "spill_count="); + settings.add_filter( + r"spilled_bytes=[\d.]+\s*[KMGTPE]?B?", + "spilled_bytes=", + ); + settings.add_filter(r"spilled_rows=\d+", "spilled_rows="); + settings.add_filter( + r"current_memory_usage=[\d.]+\s*[KMGTPE]?B?", + "current_memory_usage=", + ); + settings.add_filter( + r"start_timestamp=[\d.]+[a-zA-Zµnms]*", + "start_timestamp=", + ); + settings.add_filter( + r"end_timestamp=[\d.]+[a-zA-Zµnms]*", + "end_timestamp=", + ); + + // Common custom metric patterns + settings.add_filter(r"fetch_time=[\d.]+[a-zA-Zµnms]+", "fetch_time="); + settings.add_filter( + r"repartition_time=[\d.]+[a-zA-Zµnms]+", + "repartition_time=", + ); + settings.add_filter(r"send_time=[\d.]+[a-zA-Zµnms]+", "send_time="); + settings.add_filter(r"peak_mem_used=\d+", "peak_mem_used="); + settings.add_filter(r"batches_splitted=\d+", "batches_splitted="); + settings.add_filter(r"batches_split=\d+", "batches_split="); + settings.add_filter(r"bytes_scanned=\d+", "bytes_scanned="); + settings.add_filter(r"file_open_errors=\d+", "file_open_errors="); + settings.add_filter(r"file_scan_errors=\d+", "file_scan_errors="); + settings.add_filter( + r"files_ranges_pruned_statistics=\d+", + "files_ranges_pruned_statistics=", + ); + settings.add_filter( + r"num_predicate_creation_errors=\d+", + "num_predicate_creation_errors=", + ); + settings.add_filter( + r"page_index_rows_matched=\d+", + "page_index_rows_matched=", + ); + settings.add_filter( + r"page_index_rows_pruned=\d+", + "page_index_rows_pruned=", + ); + settings.add_filter( + r"predicate_evaluation_errors=\d+", + "predicate_evaluation_errors=", + ); + settings.add_filter( + r"pushdown_rows_matched=\d+", + "pushdown_rows_matched=", + ); + settings.add_filter(r"pushdown_rows_pruned=\d+", "pushdown_rows_pruned="); + settings.add_filter( + r"row_groups_matched_bloom_filter=\d+", + "row_groups_matched_bloom_filter=", + ); + settings.add_filter( + r"row_groups_matched_statistics=\d+", + "row_groups_matched_statistics=", + ); + settings.add_filter( + r"row_groups_pruned_bloom_filter=\d+", + "row_groups_pruned_bloom_filter=", + ); + settings.add_filter( + r"row_groups_pruned_statistics=\d+", + "row_groups_pruned_statistics=", + ); + settings.add_filter( + r"bloom_filter_eval_time=[\d.]+[a-zA-Zµnms]+", + "bloom_filter_eval_time=", + ); + settings.add_filter( + r"metadata_load_time=[\d.]+[a-zA-Zµnms]+", + "metadata_load_time=", + ); + settings.add_filter( + r"page_index_eval_time=[\d.]+[a-zA-Zµnms]+", + "page_index_eval_time=", + ); + settings.add_filter( + r"row_pushdown_eval_time=[\d.]+[a-zA-Zµnms]+", + "row_pushdown_eval_time=", + ); + settings.add_filter( + r"statistics_eval_time=[\d.]+[a-zA-Zµnms]+", + "statistics_eval_time=", + ); + settings.add_filter( + r"time_elapsed_opening=[\d.]+[a-zA-Zµnms]+", + "time_elapsed_opening=", + ); + settings.add_filter( + r"time_elapsed_processing=[\d.]+[a-zA-Zµnms]+", + "time_elapsed_processing=", + ); + settings.add_filter( + r"time_elapsed_scanning_total=[\d.]+[a-zA-Zµnms]+", + "time_elapsed_scanning_total=", + ); + settings.add_filter( + r"time_elapsed_scanning_until_data=[\d.]+[a-zA-Zµnms]+", + "time_elapsed_scanning_until_data=", + ); + settings.add_filter( + r"skipped_aggregation_rows=\d+", + "skipped_aggregation_rows=", + ); + settings.add_filter(r"build_input_batches=\d+", "build_input_batches="); + settings.add_filter(r"build_input_rows=\d+", "build_input_rows="); + 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"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="); + settings } diff --git a/tests/tpch_validation_test.rs b/tests/tpch_validation_test.rs index d31b7d3..1b4cac6 100644 --- a/tests/tpch_validation_test.rs +++ b/tests/tpch_validation_test.rs @@ -8,6 +8,7 @@ mod tests { use datafusion_distributed::test_utils::tpch; use datafusion_distributed::{ DistributedPhysicalOptimizerRule, DistributedSessionBuilderContext, assert_snapshot, + explain_analyze, }; use futures::TryStreamExt; use std::error::Error; @@ -24,7 +25,7 @@ mod tests { #[tokio::test] async fn test_tpch_1() -> Result<(), Box> { - let plan = test_tpch_query(1).await?; + let (plan, analyzed_plan) = test_tpch_query(1).await?; assert_snapshot!(plan, @r" ┌───── Stage 3 Tasks: t0:[p0] │ SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST] @@ -47,12 +48,73 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], file_type=parquet, predicate=l_shipdate@6 <= 1998-09-02, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@0 <= 1998-09-02, required_guarantees=[] └────────────────────────────────────────────────── "); + assert_snapshot!(analyzed_plan, @r" + ┌───── Stage 3 Task t0:[p0] + │ SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST], metrics=[output_rows=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 2, output_partitions=24, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 2 Task t0:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 1, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 2 Task t1:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 1, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 2 Task t2:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 1, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 2 Task t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 1, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 1 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 24), input_partitions=2, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ 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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], file_type=parquet, predicate=l_shipdate@6 <= 1998-09-02, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@0 <= 1998-09-02, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 24), input_partitions=2, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ 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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], file_type=parquet, predicate=l_shipdate@6 <= 1998-09-02, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@0 <= 1998-09-02, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 24), input_partitions=2, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ 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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], file_type=parquet, predicate=l_shipdate@6 <= 1998-09-02, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@0 <= 1998-09-02, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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(()) } #[tokio::test] async fn test_tpch_2() -> Result<(), Box> { - let plan = test_tpch_query(2).await?; + let (plan, analyzed_plan) = test_tpch_query(2).await?; assert_snapshot!(plan, @r" ┌───── Stage 8 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] @@ -135,12 +197,192 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/region/1.parquet, /testdata/tpch/data/region/10.parquet, /testdata/tpch/data/region/11.parquet], [/testdata/tpch/data/region/12.parquet, /testdata/tpch/data/region/13.parquet, /testdata/tpch/data/region/14.parquet], [/testdata/tpch/data/region/15.parquet, /testdata/tpch/data/region/16.parquet, /testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.parquet, /testdata/tpch/data/region/4.parquet, /testdata/tpch/data/region/5.parquet], [/testdata/tpch/data/region/6.parquet, /testdata/tpch/data/region/7.parquet, /testdata/tpch/data/region/8.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = EUROPE, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= EUROPE AND EUROPE <= r_name_max@1, required_guarantees=[r_name in (EUROPE)] └────────────────────────────────────────────────── "); + assert_snapshot!(analyzed_plan, @r" + ┌───── Stage 8 Task 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 7, output_partitions=24, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 7 Task t0:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] + │ 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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=1 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 6, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 7 Task t1:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] + │ 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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=1 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 6, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 7 Task t2:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] + │ 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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=1 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 6, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 7 Task t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] + │ 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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=1 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 6, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([p_partkey@0, ps_supplycost@7], 24), input_partitions=6, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 1, 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], metrics=[output_rows=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name, n_regionkey], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/supplier/1.parquet, /testdata/tpch/data/supplier/10.parquet, /testdata/tpch/data/supplier/11.parquet], [/testdata/tpch/data/supplier/12.parquet, /testdata/tpch/data/supplier/13.parquet, /testdata/tpch/data/supplier/14.parquet], [/testdata/tpch/data/supplier/15.parquet, /testdata/tpch/data/supplier/16.parquet, /testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet, /testdata/tpch/data/supplier/4.parquet, /testdata/tpch/data/supplier/5.parquet], [/testdata/tpch/data/supplier/6.parquet, /testdata/tpch/data/supplier/7.parquet, /testdata/tpch/data/supplier/8.parquet], ...]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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=, 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=, 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 2, output_partitions=8, input_tasks=4 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/partsupp/1.parquet:.., /testdata/tpch/data/partsupp/10.parquet:.., /testdata/tpch/data/partsupp/11.parquet:..], [/testdata/tpch/data/partsupp/11.parquet:.., /testdata/tpch/data/partsupp/12.parquet:.., /testdata/tpch/data/partsupp/13.parquet:.., /testdata/tpch/data/partsupp/14.parquet:..], [/testdata/tpch/data/partsupp/14.parquet:.., /testdata/tpch/data/partsupp/15.parquet:.., /testdata/tpch/data/partsupp/16.parquet:.., /testdata/tpch/data/partsupp/2.parquet:..], [/testdata/tpch/data/partsupp/2.parquet:.., /testdata/tpch/data/partsupp/3.parquet:.., /testdata/tpch/data/partsupp/4.parquet:..], [/testdata/tpch/data/partsupp/4.parquet:.., /testdata/tpch/data/partsupp/5.parquet:.., /testdata/tpch/data/partsupp/6.parquet:.., /testdata/tpch/data/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/region/1.parquet, /testdata/tpch/data/region/10.parquet, /testdata/tpch/data/region/11.parquet], [/testdata/tpch/data/region/12.parquet, /testdata/tpch/data/region/13.parquet, /testdata/tpch/data/region/14.parquet], [/testdata/tpch/data/region/15.parquet, /testdata/tpch/data/region/16.parquet, /testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.parquet, /testdata/tpch/data/region/4.parquet, /testdata/tpch/data/region/5.parquet], [/testdata/tpch/data/region/6.parquet, /testdata/tpch/data/region/7.parquet, /testdata/tpch/data/region/8.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = EUROPE, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= EUROPE AND EUROPE <= r_name_max@1, required_guarantees=[r_name in (EUROPE)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p2,p3] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/region/1.parquet, /testdata/tpch/data/region/10.parquet, /testdata/tpch/data/region/11.parquet], [/testdata/tpch/data/region/12.parquet, /testdata/tpch/data/region/13.parquet, /testdata/tpch/data/region/14.parquet], [/testdata/tpch/data/region/15.parquet, /testdata/tpch/data/region/16.parquet, /testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.parquet, /testdata/tpch/data/region/4.parquet, /testdata/tpch/data/region/5.parquet], [/testdata/tpch/data/region/6.parquet, /testdata/tpch/data/region/7.parquet, /testdata/tpch/data/region/8.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = EUROPE, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= EUROPE AND EUROPE <= r_name_max@1, required_guarantees=[r_name in (EUROPE)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/region/1.parquet, /testdata/tpch/data/region/10.parquet, /testdata/tpch/data/region/11.parquet], [/testdata/tpch/data/region/12.parquet, /testdata/tpch/data/region/13.parquet, /testdata/tpch/data/region/14.parquet], [/testdata/tpch/data/region/15.parquet, /testdata/tpch/data/region/16.parquet, /testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.parquet, /testdata/tpch/data/region/4.parquet, /testdata/tpch/data/region/5.parquet], [/testdata/tpch/data/region/6.parquet, /testdata/tpch/data/region/7.parquet, /testdata/tpch/data/region/8.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = EUROPE, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= EUROPE AND EUROPE <= r_name_max@1, required_guarantees=[r_name in (EUROPE)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/region/1.parquet, /testdata/tpch/data/region/10.parquet, /testdata/tpch/data/region/11.parquet], [/testdata/tpch/data/region/12.parquet, /testdata/tpch/data/region/13.parquet, /testdata/tpch/data/region/14.parquet], [/testdata/tpch/data/region/15.parquet, /testdata/tpch/data/region/16.parquet, /testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.parquet, /testdata/tpch/data/region/4.parquet, /testdata/tpch/data/region/5.parquet], [/testdata/tpch/data/region/6.parquet, /testdata/tpch/data/region/7.parquet, /testdata/tpch/data/region/8.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = EUROPE, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= EUROPE AND EUROPE <= r_name_max@1, required_guarantees=[r_name in (EUROPE)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@1], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_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=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p2,p3] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@1], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_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=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@1], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_mfgr, p_type, p_size], file_type=parquet, predicate=p_size@3 = 15 AND p_type@2 LIKE %BRASS, pruning_predicate=p_size_null_count@2 != row_count@3 AND p_size_min@0 <= 15 AND 15 <= p_size_max@1, required_guarantees=[p_size in (15)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@1], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_mfgr, p_type, p_size], file_type=parquet, predicate=p_size@3 = 15 AND p_type@2 LIKE %BRASS, pruning_predicate=p_size_null_count@2 != row_count@3 AND p_size_min@0 <= 15 AND 15 <= p_size_max@1, required_guarantees=[p_size in (15)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 24), input_partitions=6, metrics=[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=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 5, output_partitions=6, n_tasks=3, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 6 Task t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 24), input_partitions=6, metrics=[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=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 5, output_partitions=6, n_tasks=3, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 6 Task t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 24), input_partitions=6, metrics=[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=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 5, output_partitions=6, n_tasks=3, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 5 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([ps_partkey@0], 18), input_partitions=6, metrics=[fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 4, 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], metrics=[output_rows=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_regionkey], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/supplier/1.parquet, /testdata/tpch/data/supplier/10.parquet, /testdata/tpch/data/supplier/11.parquet], [/testdata/tpch/data/supplier/12.parquet, /testdata/tpch/data/supplier/13.parquet, /testdata/tpch/data/supplier/14.parquet], [/testdata/tpch/data/supplier/15.parquet, /testdata/tpch/data/supplier/16.parquet, /testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet, /testdata/tpch/data/supplier/4.parquet, /testdata/tpch/data/supplier/5.parquet], [/testdata/tpch/data/supplier/6.parquet, /testdata/tpch/data/supplier/7.parquet, /testdata/tpch/data/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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/data/partsupp/1.parquet:.., /testdata/tpch/data/partsupp/10.parquet:.., /testdata/tpch/data/partsupp/11.parquet:..], [/testdata/tpch/data/partsupp/11.parquet:.., /testdata/tpch/data/partsupp/12.parquet:.., /testdata/tpch/data/partsupp/13.parquet:.., /testdata/tpch/data/partsupp/14.parquet:..], [/testdata/tpch/data/partsupp/14.parquet:.., /testdata/tpch/data/partsupp/15.parquet:.., /testdata/tpch/data/partsupp/16.parquet:.., /testdata/tpch/data/partsupp/2.parquet:..], [/testdata/tpch/data/partsupp/2.parquet:.., /testdata/tpch/data/partsupp/3.parquet:.., /testdata/tpch/data/partsupp/4.parquet:..], [/testdata/tpch/data/partsupp/4.parquet:.., /testdata/tpch/data/partsupp/5.parquet:.., /testdata/tpch/data/partsupp/6.parquet:.., /testdata/tpch/data/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/region/1.parquet, /testdata/tpch/data/region/10.parquet, /testdata/tpch/data/region/11.parquet], [/testdata/tpch/data/region/12.parquet, /testdata/tpch/data/region/13.parquet, /testdata/tpch/data/region/14.parquet], [/testdata/tpch/data/region/15.parquet, /testdata/tpch/data/region/16.parquet, /testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.parquet, /testdata/tpch/data/region/4.parquet, /testdata/tpch/data/region/5.parquet], [/testdata/tpch/data/region/6.parquet, /testdata/tpch/data/region/7.parquet, /testdata/tpch/data/region/8.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = EUROPE, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= EUROPE AND EUROPE <= r_name_max@1, required_guarantees=[r_name in (EUROPE)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p2,p3] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/region/1.parquet, /testdata/tpch/data/region/10.parquet, /testdata/tpch/data/region/11.parquet], [/testdata/tpch/data/region/12.parquet, /testdata/tpch/data/region/13.parquet, /testdata/tpch/data/region/14.parquet], [/testdata/tpch/data/region/15.parquet, /testdata/tpch/data/region/16.parquet, /testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.parquet, /testdata/tpch/data/region/4.parquet, /testdata/tpch/data/region/5.parquet], [/testdata/tpch/data/region/6.parquet, /testdata/tpch/data/region/7.parquet, /testdata/tpch/data/region/8.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = EUROPE, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= EUROPE AND EUROPE <= r_name_max@1, required_guarantees=[r_name in (EUROPE)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/region/1.parquet, /testdata/tpch/data/region/10.parquet, /testdata/tpch/data/region/11.parquet], [/testdata/tpch/data/region/12.parquet, /testdata/tpch/data/region/13.parquet, /testdata/tpch/data/region/14.parquet], [/testdata/tpch/data/region/15.parquet, /testdata/tpch/data/region/16.parquet, /testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.parquet, /testdata/tpch/data/region/4.parquet, /testdata/tpch/data/region/5.parquet], [/testdata/tpch/data/region/6.parquet, /testdata/tpch/data/region/7.parquet, /testdata/tpch/data/region/8.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = EUROPE, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= EUROPE AND EUROPE <= r_name_max@1, required_guarantees=[r_name in (EUROPE)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/region/1.parquet, /testdata/tpch/data/region/10.parquet, /testdata/tpch/data/region/11.parquet], [/testdata/tpch/data/region/12.parquet, /testdata/tpch/data/region/13.parquet, /testdata/tpch/data/region/14.parquet], [/testdata/tpch/data/region/15.parquet, /testdata/tpch/data/region/16.parquet, /testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.parquet, /testdata/tpch/data/region/4.parquet, /testdata/tpch/data/region/5.parquet], [/testdata/tpch/data/region/6.parquet, /testdata/tpch/data/region/7.parquet, /testdata/tpch/data/region/8.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = EUROPE, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= EUROPE AND EUROPE <= r_name_max@1, required_guarantees=[r_name in (EUROPE)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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(()) } #[tokio::test] async fn test_tpch_3() -> Result<(), Box> { - let plan = test_tpch_query(3).await?; + let (plan, analyzed_plan) = test_tpch_query(3).await?; assert_snapshot!(plan, @r" ┌───── Stage 4 Tasks: t0:[p0] │ SortPreservingMergeExec: [revenue@1 DESC, o_orderdate@2 ASC NULLS LAST] @@ -177,12 +419,87 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_mktsegment], file_type=parquet, predicate=c_mktsegment@1 = BUILDING, pruning_predicate=c_mktsegment_null_count@2 != row_count@3 AND c_mktsegment_min@0 <= BUILDING AND BUILDING <= c_mktsegment_max@1, required_guarantees=[c_mktsegment in (BUILDING)] └────────────────────────────────────────────────── "); + assert_snapshot!(analyzed_plan, @r" + ┌───── Stage 4 Task t0:[p0] + │ SortPreservingMergeExec: [revenue@1 DESC, o_orderdate@2 ASC NULLS LAST], metrics=[output_rows=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 3, output_partitions=24, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t0:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[revenue@1 DESC, o_orderdate@2 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t1:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[revenue@1 DESC, o_orderdate@2 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t2:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[revenue@1 DESC, o_orderdate@2 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[revenue@1 DESC, o_orderdate@2 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 2 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([l_orderkey@0, o_orderdate@1, o_shippriority@2], 24), input_partitions=6, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 1, output_partitions=8, input_tasks=4 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: o_orderdate@2 < 1995-03-15, metrics=[output_rows=, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], file_type=parquet, predicate=o_orderdate@2 < 1995-03-15 AND DynamicFilterPhysicalExpr [ true ], pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@0 < 1995-03-15, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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=, elapsed_compute=] + │ FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2], metrics=[output_rows=, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 > 1995-03-15 AND DynamicFilterPhysicalExpr [ true ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 > 1995-03-15, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_mktsegment], file_type=parquet, predicate=c_mktsegment@1 = BUILDING, pruning_predicate=c_mktsegment_null_count@2 != row_count@3 AND c_mktsegment_min@0 <= BUILDING AND BUILDING <= c_mktsegment_max@1, required_guarantees=[c_mktsegment in (BUILDING)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p2,p3] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_mktsegment], file_type=parquet, predicate=c_mktsegment@1 = BUILDING, pruning_predicate=c_mktsegment_null_count@2 != row_count@3 AND c_mktsegment_min@0 <= BUILDING AND BUILDING <= c_mktsegment_max@1, required_guarantees=[c_mktsegment in (BUILDING)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_mktsegment], file_type=parquet, predicate=c_mktsegment@1 = BUILDING, pruning_predicate=c_mktsegment_null_count@2 != row_count@3 AND c_mktsegment_min@0 <= BUILDING AND BUILDING <= c_mktsegment_max@1, required_guarantees=[c_mktsegment in (BUILDING)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_mktsegment], file_type=parquet, predicate=c_mktsegment@1 = BUILDING, pruning_predicate=c_mktsegment_null_count@2 != row_count@3 AND c_mktsegment_min@0 <= BUILDING AND BUILDING <= c_mktsegment_max@1, required_guarantees=[c_mktsegment in (BUILDING)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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(()) } #[tokio::test] async fn test_tpch_4() -> Result<(), Box> { - let plan = test_tpch_query(4).await?; + let (plan, analyzed_plan) = test_tpch_query(4).await?; assert_snapshot!(plan, @r" ┌───── Stage 4 Tasks: t0:[p0] │ SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST] @@ -213,12 +530,81 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@2 > l_commitdate@1 └────────────────────────────────────────────────── "); + assert_snapshot!(analyzed_plan, @r" + ┌───── Stage 4 Task t0:[p0] + │ SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST], metrics=[output_rows=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 3, output_partitions=24, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t0:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[o_orderpriority@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t1:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[o_orderpriority@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t2:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[o_orderpriority@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[o_orderpriority@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 2 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([o_orderpriority@0], 24), input_partitions=6, metrics=[fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(l_orderkey@0, o_orderkey@0)], projection=[o_orderpriority@1], metrics=[output_rows=, 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 1, output_partitions=8, input_tasks=4 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderdate, o_orderpriority], file_type=parquet, predicate=o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1993-07-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 < 1993-10-01, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@2 > l_commitdate@1, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p2,p3] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@2 > l_commitdate@1, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@2 > l_commitdate@1, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@2 > l_commitdate@1, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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(()) } #[tokio::test] async fn test_tpch_5() -> Result<(), Box> { - let plan = test_tpch_query(5).await?; + let (plan, analyzed_plan) = test_tpch_query(5).await?; assert_snapshot!(plan, @r" ┌───── Stage 5 Tasks: t0:[p0] │ SortPreservingMergeExec: [revenue@1 DESC] @@ -272,12 +658,122 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1994-01-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 < 1995-01-01, required_guarantees=[] └────────────────────────────────────────────────── "); + assert_snapshot!(analyzed_plan, @r" + ┌───── Stage 5 Task t0:[p0] + │ SortPreservingMergeExec: [revenue@1 DESC], metrics=[output_rows=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 4, output_partitions=24, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t0:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[revenue@1 DESC], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t1:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[revenue@1 DESC], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t2:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[revenue@1 DESC], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[revenue@1 DESC], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([n_name@0], 24), input_partitions=6, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 1, 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], metrics=[output_rows=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name, n_regionkey], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/supplier/1.parquet, /testdata/tpch/data/supplier/10.parquet, /testdata/tpch/data/supplier/11.parquet], [/testdata/tpch/data/supplier/12.parquet, /testdata/tpch/data/supplier/13.parquet, /testdata/tpch/data/supplier/14.parquet], [/testdata/tpch/data/supplier/15.parquet, /testdata/tpch/data/supplier/16.parquet, /testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet, /testdata/tpch/data/supplier/4.parquet, /testdata/tpch/data/supplier/5.parquet], [/testdata/tpch/data/supplier/6.parquet, /testdata/tpch/data/supplier/7.parquet, /testdata/tpch/data/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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=, 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=, 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=, elapsed_compute=] + │ ProjectionExec: expr=[c_nationkey@1 as c_nationkey, o_orderkey@0 as o_orderkey], metrics=[output_rows=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 2, output_partitions=8, input_tasks=4 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/region/1.parquet, /testdata/tpch/data/region/10.parquet, /testdata/tpch/data/region/11.parquet], [/testdata/tpch/data/region/12.parquet, /testdata/tpch/data/region/13.parquet, /testdata/tpch/data/region/14.parquet], [/testdata/tpch/data/region/15.parquet, /testdata/tpch/data/region/16.parquet, /testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.parquet, /testdata/tpch/data/region/4.parquet, /testdata/tpch/data/region/5.parquet], [/testdata/tpch/data/region/6.parquet, /testdata/tpch/data/region/7.parquet, /testdata/tpch/data/region/8.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = ASIA, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= ASIA AND ASIA <= r_name_max@1, required_guarantees=[r_name in (ASIA)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p2,p3] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/region/1.parquet, /testdata/tpch/data/region/10.parquet, /testdata/tpch/data/region/11.parquet], [/testdata/tpch/data/region/12.parquet, /testdata/tpch/data/region/13.parquet, /testdata/tpch/data/region/14.parquet], [/testdata/tpch/data/region/15.parquet, /testdata/tpch/data/region/16.parquet, /testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.parquet, /testdata/tpch/data/region/4.parquet, /testdata/tpch/data/region/5.parquet], [/testdata/tpch/data/region/6.parquet, /testdata/tpch/data/region/7.parquet, /testdata/tpch/data/region/8.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = ASIA, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= ASIA AND ASIA <= r_name_max@1, required_guarantees=[r_name in (ASIA)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/region/1.parquet, /testdata/tpch/data/region/10.parquet, /testdata/tpch/data/region/11.parquet], [/testdata/tpch/data/region/12.parquet, /testdata/tpch/data/region/13.parquet, /testdata/tpch/data/region/14.parquet], [/testdata/tpch/data/region/15.parquet, /testdata/tpch/data/region/16.parquet, /testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.parquet, /testdata/tpch/data/region/4.parquet, /testdata/tpch/data/region/5.parquet], [/testdata/tpch/data/region/6.parquet, /testdata/tpch/data/region/7.parquet, /testdata/tpch/data/region/8.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = ASIA, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= ASIA AND ASIA <= r_name_max@1, required_guarantees=[r_name in (ASIA)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/region/1.parquet, /testdata/tpch/data/region/10.parquet, /testdata/tpch/data/region/11.parquet], [/testdata/tpch/data/region/12.parquet, /testdata/tpch/data/region/13.parquet, /testdata/tpch/data/region/14.parquet], [/testdata/tpch/data/region/15.parquet, /testdata/tpch/data/region/16.parquet, /testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.parquet, /testdata/tpch/data/region/4.parquet, /testdata/tpch/data/region/5.parquet], [/testdata/tpch/data/region/6.parquet, /testdata/tpch/data/region/7.parquet, /testdata/tpch/data/region/8.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = ASIA, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= ASIA AND ASIA <= r_name_max@1, required_guarantees=[r_name in (ASIA)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1994-01-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 < 1995-01-01, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p2,p3] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1994-01-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 < 1995-01-01, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1994-01-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 < 1995-01-01, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1994-01-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 < 1995-01-01, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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(()) } #[tokio::test] async fn test_tpch_6() -> Result<(), Box> { - let plan = test_tpch_query(6).await?; + let (plan, analyzed_plan) = test_tpch_query(6).await?; assert_snapshot!(plan, @r" ┌───── Stage 2 Tasks: t0:[p0] │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * lineitem.l_discount)@0 as revenue] @@ -293,12 +789,48 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1994-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1995-01-01 AND l_discount_null_count@5 != row_count@2 AND l_discount_max@4 >= Some(5),15,2 AND l_discount_null_count@5 != row_count@2 AND l_discount_min@6 <= Some(7),15,2 AND l_quantity_null_count@8 != row_count@2 AND l_quantity_min@7 < Some(2400),15,2, required_guarantees=[] └────────────────────────────────────────────────── "); + assert_snapshot!(analyzed_plan, @r" + ┌───── Stage 2 Task t0:[p0] + │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * lineitem.l_discount)@0 as revenue], metrics=[output_rows=, elapsed_compute=] + │ AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)], metrics=[output_rows=, elapsed_compute=] + │ CoalescePartitionsExec, metrics=[output_rows=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 1, output_partitions=8, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 1 Task t0:[p0,p1] + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)], metrics=[output_rows=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2, projection=[l_extendedprice@1, l_discount@2], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1994-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1995-01-01 AND l_discount_null_count@5 != row_count@2 AND l_discount_max@4 >= Some(5),15,2 AND l_discount_null_count@5 != row_count@2 AND l_discount_min@6 <= Some(7),15,2 AND l_quantity_null_count@8 != row_count@2 AND l_quantity_min@7 < Some(2400),15,2, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p2,p3] + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)], metrics=[output_rows=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2, projection=[l_extendedprice@1, l_discount@2], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1994-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1995-01-01 AND l_discount_null_count@5 != row_count@2 AND l_discount_max@4 >= Some(5),15,2 AND l_discount_null_count@5 != row_count@2 AND l_discount_min@6 <= Some(7),15,2 AND l_quantity_null_count@8 != row_count@2 AND l_quantity_min@7 < Some(2400),15,2, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p4,p5] + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)], metrics=[output_rows=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2, projection=[l_extendedprice@1, l_discount@2], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1994-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1995-01-01 AND l_discount_null_count@5 != row_count@2 AND l_discount_max@4 >= Some(5),15,2 AND l_discount_null_count@5 != row_count@2 AND l_discount_min@6 <= Some(7),15,2 AND l_quantity_null_count@8 != row_count@2 AND l_quantity_min@7 < Some(2400),15,2, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t3:[p6,p7] + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)], metrics=[output_rows=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2, projection=[l_extendedprice@1, l_discount@2], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1994-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1995-01-01 AND l_discount_null_count@5 != row_count@2 AND l_discount_max@4 >= Some(5),15,2 AND l_discount_null_count@5 != row_count@2 AND l_discount_min@6 <= Some(7),15,2 AND l_quantity_null_count@8 != row_count@2 AND l_quantity_min@7 < Some(2400),15,2, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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(()) } #[tokio::test] async fn test_tpch_7() -> Result<(), Box> { - let plan = test_tpch_query(7).await?; + let (plan, analyzed_plan) = test_tpch_query(7).await?; assert_snapshot!(plan, @r" ┌───── Stage 9 Tasks: t0:[p0] │ SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST] @@ -374,12 +906,182 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── "); + assert_snapshot!(analyzed_plan, @r" + ┌───── Stage 9 Task t0:[p0] + │ SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST], metrics=[output_rows=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 8, output_partitions=24, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 8 Task t0:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 7, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 8 Task t1:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 7, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 8 Task t2:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 7, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 8 Task t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 7, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 7 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 24), input_partitions=6, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ 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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 1, 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], metrics=[output_rows=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 2, 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], metrics=[output_rows=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=1, input_tasks=3 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 6, output_partitions=6, n_tasks=1, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 1 Task t0:[p0,p1] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE, metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = GERMANY OR n_name@1 = FRANCE, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1 OR n_name_null_count@2 != row_count@3 AND n_name_min@0 <= FRANCE AND FRANCE <= n_name_max@1, required_guarantees=[n_name in (FRANCE, GERMANY)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p2,p3] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE, metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = GERMANY OR n_name@1 = FRANCE, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1 OR n_name_null_count@2 != row_count@3 AND n_name_min@0 <= FRANCE AND FRANCE <= n_name_max@1, required_guarantees=[n_name in (FRANCE, GERMANY)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE, metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = GERMANY OR n_name@1 = FRANCE, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1 OR n_name_null_count@2 != row_count@3 AND n_name_min@0 <= FRANCE AND FRANCE <= n_name_max@1, required_guarantees=[n_name in (FRANCE, GERMANY)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE, metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = GERMANY OR n_name@1 = FRANCE, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1 OR n_name_null_count@2 != row_count@3 AND n_name_min@0 <= FRANCE AND FRANCE <= n_name_max@1, required_guarantees=[n_name in (FRANCE, GERMANY)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY, metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = FRANCE OR n_name@1 = GERMANY, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= FRANCE AND FRANCE <= n_name_max@1 OR n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1, required_guarantees=[n_name in (FRANCE, GERMANY)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p2,p3] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY, metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = FRANCE OR n_name@1 = GERMANY, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= FRANCE AND FRANCE <= n_name_max@1 OR n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1, required_guarantees=[n_name in (FRANCE, GERMANY)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY, metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = FRANCE OR n_name@1 = GERMANY, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= FRANCE AND FRANCE <= n_name_max@1 OR n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1, required_guarantees=[n_name in (FRANCE, GERMANY)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY, metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = FRANCE OR n_name@1 = GERMANY, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= FRANCE AND FRANCE <= n_name_max@1 OR n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1, required_guarantees=[n_name in (FRANCE, GERMANY)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([c_custkey@0], 6), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([c_custkey@0], 6), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([c_custkey@0], 6), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([o_custkey@4], 6), input_partitions=6, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 4, output_partitions=6, n_tasks=3, input_tasks=1 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 5, output_partitions=6, n_tasks=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 6 Task t1:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([o_custkey@4], 6), input_partitions=6, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 4, output_partitions=6, n_tasks=3, input_tasks=1 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 5, output_partitions=6, n_tasks=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 6 Task t2:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([o_custkey@4], 6), input_partitions=6, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 4, output_partitions=6, n_tasks=3, input_tasks=1 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 5, output_partitions=6, n_tasks=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([l_orderkey@1], 18), input_partitions=6, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/supplier/1.parquet, /testdata/tpch/data/supplier/10.parquet, /testdata/tpch/data/supplier/11.parquet], [/testdata/tpch/data/supplier/12.parquet, /testdata/tpch/data/supplier/13.parquet, /testdata/tpch/data/supplier/14.parquet], [/testdata/tpch/data/supplier/15.parquet, /testdata/tpch/data/supplier/16.parquet, /testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet, /testdata/tpch/data/supplier/4.parquet, /testdata/tpch/data/supplier/5.parquet], [/testdata/tpch/data/supplier/6.parquet, /testdata/tpch/data/supplier/7.parquet, /testdata/tpch/data/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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=, elapsed_compute=] + │ FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31, metrics=[output_rows=, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 AND DynamicFilterPhysicalExpr [ true ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1995-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 <= 1996-12-31, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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(()) } #[tokio::test] async fn test_tpch_8() -> Result<(), Box> { - let plan = test_tpch_query(8).await?; + let (plan, analyzed_plan) = test_tpch_query(8).await?; assert_snapshot!(plan, @r#" ┌───── Stage 9 Tasks: t0:[p0] │ SortPreservingMergeExec: [o_year@0 ASC NULLS LAST] @@ -465,12 +1167,198 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── "#); + assert_snapshot!(analyzed_plan, @r#" + ┌───── Stage 9 Task t0:[p0] + │ SortPreservingMergeExec: [o_year@0 ASC NULLS LAST], metrics=[output_rows=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 8, output_partitions=24, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 8 Task t0:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[o_year@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 7, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 8 Task t1:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[o_year@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 7, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 8 Task t2:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[o_year@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 7, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 8 Task t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[o_year@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 7, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 7 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([o_year@0], 24), input_partitions=6, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ 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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 1, 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], metrics=[output_rows=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_regionkey], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 5, output_partitions=6, n_tasks=1, input_tasks=3 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 6, output_partitions=6, n_tasks=1, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 1 Task t0:[p0,p1] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/region/1.parquet, /testdata/tpch/data/region/10.parquet, /testdata/tpch/data/region/11.parquet], [/testdata/tpch/data/region/12.parquet, /testdata/tpch/data/region/13.parquet, /testdata/tpch/data/region/14.parquet], [/testdata/tpch/data/region/15.parquet, /testdata/tpch/data/region/16.parquet, /testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.parquet, /testdata/tpch/data/region/4.parquet, /testdata/tpch/data/region/5.parquet], [/testdata/tpch/data/region/6.parquet, /testdata/tpch/data/region/7.parquet, /testdata/tpch/data/region/8.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = AMERICA, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= AMERICA AND AMERICA <= r_name_max@1, required_guarantees=[r_name in (AMERICA)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p2,p3] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/region/1.parquet, /testdata/tpch/data/region/10.parquet, /testdata/tpch/data/region/11.parquet], [/testdata/tpch/data/region/12.parquet, /testdata/tpch/data/region/13.parquet, /testdata/tpch/data/region/14.parquet], [/testdata/tpch/data/region/15.parquet, /testdata/tpch/data/region/16.parquet, /testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.parquet, /testdata/tpch/data/region/4.parquet, /testdata/tpch/data/region/5.parquet], [/testdata/tpch/data/region/6.parquet, /testdata/tpch/data/region/7.parquet, /testdata/tpch/data/region/8.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = AMERICA, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= AMERICA AND AMERICA <= r_name_max@1, required_guarantees=[r_name in (AMERICA)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/region/1.parquet, /testdata/tpch/data/region/10.parquet, /testdata/tpch/data/region/11.parquet], [/testdata/tpch/data/region/12.parquet, /testdata/tpch/data/region/13.parquet, /testdata/tpch/data/region/14.parquet], [/testdata/tpch/data/region/15.parquet, /testdata/tpch/data/region/16.parquet, /testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.parquet, /testdata/tpch/data/region/4.parquet, /testdata/tpch/data/region/5.parquet], [/testdata/tpch/data/region/6.parquet, /testdata/tpch/data/region/7.parquet, /testdata/tpch/data/region/8.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = AMERICA, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= AMERICA AND AMERICA <= r_name_max@1, required_guarantees=[r_name in (AMERICA)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/region/1.parquet, /testdata/tpch/data/region/10.parquet, /testdata/tpch/data/region/11.parquet], [/testdata/tpch/data/region/12.parquet, /testdata/tpch/data/region/13.parquet, /testdata/tpch/data/region/14.parquet], [/testdata/tpch/data/region/15.parquet, /testdata/tpch/data/region/16.parquet, /testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.parquet, /testdata/tpch/data/region/4.parquet, /testdata/tpch/data/region/5.parquet], [/testdata/tpch/data/region/6.parquet, /testdata/tpch/data/region/7.parquet, /testdata/tpch/data/region/8.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = AMERICA, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= AMERICA AND AMERICA <= r_name_max@1, required_guarantees=[r_name in (AMERICA)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([o_custkey@3], 6), input_partitions=6, metrics=[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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=3, input_tasks=3 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 4, output_partitions=6, n_tasks=3, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 5 Task t1:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([o_custkey@3], 6), input_partitions=6, metrics=[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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=3, input_tasks=3 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 4, output_partitions=6, n_tasks=3, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 5 Task t2:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([o_custkey@3], 6), input_partitions=6, metrics=[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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=3, input_tasks=3 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 4, output_partitions=6, n_tasks=3, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 2 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31, metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31, pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1995-01-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 <= 1996-12-31, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31, metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31, pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1995-01-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 <= 1996-12-31, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31, metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31, pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1995-01-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 <= 1996-12-31, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 18), input_partitions=6, metrics=[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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/supplier/1.parquet, /testdata/tpch/data/supplier/10.parquet, /testdata/tpch/data/supplier/11.parquet], [/testdata/tpch/data/supplier/12.parquet, /testdata/tpch/data/supplier/13.parquet, /testdata/tpch/data/supplier/14.parquet], [/testdata/tpch/data/supplier/15.parquet, /testdata/tpch/data/supplier/16.parquet, /testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet, /testdata/tpch/data/supplier/4.parquet, /testdata/tpch/data/supplier/5.parquet], [/testdata/tpch/data/supplier/6.parquet, /testdata/tpch/data/supplier/7.parquet, /testdata/tpch/data/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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=, 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=, 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 3, output_partitions=8, input_tasks=4 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: p_type@1 = ECONOMY ANODIZED STEEL, projection=[p_partkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_type], file_type=parquet, 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=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p2,p3] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: p_type@1 = ECONOMY ANODIZED STEEL, projection=[p_partkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_type], file_type=parquet, 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=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: p_type@1 = ECONOMY ANODIZED STEEL, projection=[p_partkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_type], file_type=parquet, predicate=p_type@1 = ECONOMY ANODIZED STEEL, pruning_predicate=p_type_null_count@2 != row_count@3 AND p_type_min@0 <= ECONOMY ANODIZED STEEL AND ECONOMY ANODIZED STEEL <= p_type_max@1, required_guarantees=[p_type in (ECONOMY ANODIZED STEEL)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: p_type@1 = ECONOMY ANODIZED STEEL, projection=[p_partkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_type], file_type=parquet, predicate=p_type@1 = ECONOMY ANODIZED STEEL, pruning_predicate=p_type_null_count@2 != row_count@3 AND p_type_min@0 <= ECONOMY ANODIZED STEEL AND ECONOMY ANODIZED STEEL <= p_type_max@1, required_guarantees=[p_type in (ECONOMY ANODIZED STEEL)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([c_custkey@0], 6), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([c_custkey@0], 6), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([c_custkey@0], 6), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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(()) } #[tokio::test] async fn test_tpch_9() -> Result<(), Box> { - let plan = test_tpch_query(9).await?; + let (plan, analyzed_plan) = test_tpch_query(9).await?; assert_snapshot!(plan, @r" ┌───── Stage 8 Tasks: t0:[p0] │ SortPreservingMergeExec: [nation@0 ASC NULLS LAST, o_year@1 DESC] @@ -538,12 +1426,156 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/partsupp/1.parquet:.., /testdata/tpch/data/partsupp/10.parquet:.., /testdata/tpch/data/partsupp/11.parquet:..], [/testdata/tpch/data/partsupp/11.parquet:.., /testdata/tpch/data/partsupp/12.parquet:.., /testdata/tpch/data/partsupp/13.parquet:.., /testdata/tpch/data/partsupp/14.parquet:..], [/testdata/tpch/data/partsupp/14.parquet:.., /testdata/tpch/data/partsupp/15.parquet:.., /testdata/tpch/data/partsupp/16.parquet:.., /testdata/tpch/data/partsupp/2.parquet:..], [/testdata/tpch/data/partsupp/2.parquet:.., /testdata/tpch/data/partsupp/3.parquet:.., /testdata/tpch/data/partsupp/4.parquet:..], [/testdata/tpch/data/partsupp/4.parquet:.., /testdata/tpch/data/partsupp/5.parquet:.., /testdata/tpch/data/partsupp/6.parquet:.., /testdata/tpch/data/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── "); + assert_snapshot!(analyzed_plan, @r" + ┌───── Stage 8 Task t0:[p0] + │ SortPreservingMergeExec: [nation@0 ASC NULLS LAST, o_year@1 DESC], metrics=[output_rows=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 7, output_partitions=24, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 7 Task t0:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[nation@0 ASC NULLS LAST, o_year@1 DESC], preserve_partitioning=[true], metrics=[output_rows=, 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=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 6, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 7 Task t1:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[nation@0 ASC NULLS LAST, o_year@1 DESC], preserve_partitioning=[true], metrics=[output_rows=, 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=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 6, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 7 Task t2:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[nation@0 ASC NULLS LAST, o_year@1 DESC], preserve_partitioning=[true], metrics=[output_rows=, 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=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 6, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 7 Task t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[nation@0 ASC NULLS LAST, o_year@1 DESC], preserve_partitioning=[true], metrics=[output_rows=, 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=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 6, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 6 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([nation@0, o_year@1], 24), input_partitions=6, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ 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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 1, output_partitions=6, n_tasks=1, input_tasks=3 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 5, output_partitions=6, n_tasks=1, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 1 Task t0:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 6), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderdate], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 6), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderdate], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 6), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderdate], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 6), input_partitions=6, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=3, input_tasks=1 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 4, output_partitions=6, n_tasks=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 5 Task t1:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 6), input_partitions=6, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=3, input_tasks=1 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 4, output_partitions=6, n_tasks=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 5 Task t2:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 6), input_partitions=6, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=3, input_tasks=1 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 4, output_partitions=6, n_tasks=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([l_suppkey@2, l_partkey@1], 18), input_partitions=6, metrics=[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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/supplier/1.parquet, /testdata/tpch/data/supplier/10.parquet, /testdata/tpch/data/supplier/11.parquet], [/testdata/tpch/data/supplier/12.parquet, /testdata/tpch/data/supplier/13.parquet, /testdata/tpch/data/supplier/14.parquet], [/testdata/tpch/data/supplier/15.parquet, /testdata/tpch/data/supplier/16.parquet, /testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet, /testdata/tpch/data/supplier/4.parquet, /testdata/tpch/data/supplier/5.parquet], [/testdata/tpch/data/supplier/6.parquet, /testdata/tpch/data/supplier/7.parquet, /testdata/tpch/data/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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=, 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=, 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 2, output_partitions=8, input_tasks=4 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: p_name@1 LIKE %green%, projection=[p_partkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_name], file_type=parquet, predicate=p_name@1 LIKE %green%, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p2,p3] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: p_name@1 LIKE %green%, projection=[p_partkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_name], file_type=parquet, predicate=p_name@1 LIKE %green%, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: p_name@1 LIKE %green%, projection=[p_partkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_name], file_type=parquet, predicate=p_name@1 LIKE %green%, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: p_name@1 LIKE %green%, projection=[p_partkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_name], file_type=parquet, predicate=p_name@1 LIKE %green%, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/partsupp/1.parquet:.., /testdata/tpch/data/partsupp/10.parquet:.., /testdata/tpch/data/partsupp/11.parquet:..], [/testdata/tpch/data/partsupp/11.parquet:.., /testdata/tpch/data/partsupp/12.parquet:.., /testdata/tpch/data/partsupp/13.parquet:.., /testdata/tpch/data/partsupp/14.parquet:..], [/testdata/tpch/data/partsupp/14.parquet:.., /testdata/tpch/data/partsupp/15.parquet:.., /testdata/tpch/data/partsupp/16.parquet:.., /testdata/tpch/data/partsupp/2.parquet:..], [/testdata/tpch/data/partsupp/2.parquet:.., /testdata/tpch/data/partsupp/3.parquet:.., /testdata/tpch/data/partsupp/4.parquet:..], [/testdata/tpch/data/partsupp/4.parquet:.., /testdata/tpch/data/partsupp/5.parquet:.., /testdata/tpch/data/partsupp/6.parquet:.., /testdata/tpch/data/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/partsupp/1.parquet:.., /testdata/tpch/data/partsupp/10.parquet:.., /testdata/tpch/data/partsupp/11.parquet:..], [/testdata/tpch/data/partsupp/11.parquet:.., /testdata/tpch/data/partsupp/12.parquet:.., /testdata/tpch/data/partsupp/13.parquet:.., /testdata/tpch/data/partsupp/14.parquet:..], [/testdata/tpch/data/partsupp/14.parquet:.., /testdata/tpch/data/partsupp/15.parquet:.., /testdata/tpch/data/partsupp/16.parquet:.., /testdata/tpch/data/partsupp/2.parquet:..], [/testdata/tpch/data/partsupp/2.parquet:.., /testdata/tpch/data/partsupp/3.parquet:.., /testdata/tpch/data/partsupp/4.parquet:..], [/testdata/tpch/data/partsupp/4.parquet:.., /testdata/tpch/data/partsupp/5.parquet:.., /testdata/tpch/data/partsupp/6.parquet:.., /testdata/tpch/data/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/partsupp/1.parquet:.., /testdata/tpch/data/partsupp/10.parquet:.., /testdata/tpch/data/partsupp/11.parquet:..], [/testdata/tpch/data/partsupp/11.parquet:.., /testdata/tpch/data/partsupp/12.parquet:.., /testdata/tpch/data/partsupp/13.parquet:.., /testdata/tpch/data/partsupp/14.parquet:..], [/testdata/tpch/data/partsupp/14.parquet:.., /testdata/tpch/data/partsupp/15.parquet:.., /testdata/tpch/data/partsupp/16.parquet:.., /testdata/tpch/data/partsupp/2.parquet:..], [/testdata/tpch/data/partsupp/2.parquet:.., /testdata/tpch/data/partsupp/3.parquet:.., /testdata/tpch/data/partsupp/4.parquet:..], [/testdata/tpch/data/partsupp/4.parquet:.., /testdata/tpch/data/partsupp/5.parquet:.., /testdata/tpch/data/partsupp/6.parquet:.., /testdata/tpch/data/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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(()) } #[tokio::test] async fn test_tpch_10() -> Result<(), Box> { - let plan = test_tpch_query(10).await?; + let (plan, analyzed_plan) = test_tpch_query(10).await?; assert_snapshot!(plan, @r" ┌───── Stage 4 Tasks: t0:[p0] │ SortPreservingMergeExec: [revenue@2 DESC] @@ -584,12 +1616,91 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1993-10-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 < 1994-01-01, required_guarantees=[] └────────────────────────────────────────────────── "); + assert_snapshot!(analyzed_plan, @r" + ┌───── Stage 4 Task t0:[p0] + │ SortPreservingMergeExec: [revenue@2 DESC], metrics=[output_rows=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 3, output_partitions=24, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t0:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[revenue@2 DESC], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t1:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[revenue@2 DESC], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t2:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[revenue@2 DESC], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[revenue@2 DESC], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 2 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([c_custkey@0, c_name@1, c_acctbal@2, c_phone@3, n_name@4, c_address@5, c_comment@6], 24), input_partitions=6, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ 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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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=, 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=, 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=, 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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 1, output_partitions=8, input_tasks=4 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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=, elapsed_compute=] + │ FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2], metrics=[output_rows=, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], file_type=parquet, predicate=l_returnflag@3 = R AND DynamicFilterPhysicalExpr [ true ], pruning_predicate=l_returnflag_null_count@2 != row_count@3 AND l_returnflag_min@0 <= R AND R <= l_returnflag_max@1, required_guarantees=[l_returnflag in (R)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1993-10-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 < 1994-01-01, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p2,p3] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1993-10-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 < 1994-01-01, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1993-10-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 < 1994-01-01, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1993-10-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 < 1994-01-01, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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(()) } #[tokio::test] async fn test_tpch_11() -> Result<(), Box> { - let plan = test_tpch_query(11).await?; + let (plan, analyzed_plan) = test_tpch_query(11).await?; assert_snapshot!(plan, @r" ┌───── Stage 3 Tasks: t0:[p0] │ SortPreservingMergeExec: [value@1 DESC] @@ -638,12 +1749,96 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = GERMANY, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1, required_guarantees=[n_name in (GERMANY)] └────────────────────────────────────────────────── "); + assert_snapshot!(analyzed_plan, @r" + ┌───── Stage 3 Task t0:[p0] + │ SortPreservingMergeExec: [value@1 DESC], metrics=[output_rows=, elapsed_compute=] + │ SortExec: expr=[value@1 DESC], preserve_partitioning=[true], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] + │ ProjectionExec: expr=[ps_partkey@0 as ps_partkey, sum(partsupp.ps_supplycost * partsupp.ps_availqty)@1 as value], metrics=[output_rows=, elapsed_compute=] + │ NestedLoopJoinExec: join_type=Inner, filter=CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Decimal128(38, 15)) > sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)@1, projection=[ps_partkey@1, sum(partsupp.ps_supplycost * partsupp.ps_availqty)@2], metrics=[output_rows=, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ 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=, elapsed_compute=] + │ AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)], metrics=[output_rows=, elapsed_compute=] + │ CoalescePartitionsExec, metrics=[output_rows=, elapsed_compute=] + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)], metrics=[output_rows=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[ps_availqty@1, ps_supplycost@2], metrics=[output_rows=, 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 1, output_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost, s_nationkey@0 as s_nationkey], metrics=[output_rows=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ 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=, 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=, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/supplier/1.parquet, /testdata/tpch/data/supplier/10.parquet, /testdata/tpch/data/supplier/11.parquet], [/testdata/tpch/data/supplier/12.parquet, /testdata/tpch/data/supplier/13.parquet, /testdata/tpch/data/supplier/14.parquet], [/testdata/tpch/data/supplier/15.parquet, /testdata/tpch/data/supplier/16.parquet, /testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet, /testdata/tpch/data/supplier/4.parquet, /testdata/tpch/data/supplier/5.parquet], [/testdata/tpch/data/supplier/6.parquet, /testdata/tpch/data/supplier/7.parquet, /testdata/tpch/data/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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/data/partsupp/1.parquet:.., /testdata/tpch/data/partsupp/10.parquet:.., /testdata/tpch/data/partsupp/11.parquet:..], [/testdata/tpch/data/partsupp/11.parquet:.., /testdata/tpch/data/partsupp/12.parquet:.., /testdata/tpch/data/partsupp/13.parquet:.., /testdata/tpch/data/partsupp/14.parquet:..], [/testdata/tpch/data/partsupp/14.parquet:.., /testdata/tpch/data/partsupp/15.parquet:.., /testdata/tpch/data/partsupp/16.parquet:.., /testdata/tpch/data/partsupp/2.parquet:..], [/testdata/tpch/data/partsupp/2.parquet:.., /testdata/tpch/data/partsupp/3.parquet:.., /testdata/tpch/data/partsupp/4.parquet:..], [/testdata/tpch/data/partsupp/4.parquet:.., /testdata/tpch/data/partsupp/5.parquet:.., /testdata/tpch/data/partsupp/6.parquet:.., /testdata/tpch/data/partsupp/7.parquet:..], ...]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ ps_suppkey@0 >= 1 AND ps_suppkey@0 <= 1000 OR ps_suppkey@0 >= 1 AND ps_suppkey@0 <= 1000 OR ps_suppkey@0 >= 1 AND ps_suppkey@0 <= 1000 OR ps_suppkey@0 >= 1 AND ps_suppkey@0 <= 1000 OR ps_suppkey@0 >= 1 AND ps_suppkey@0 <= 1000 OR 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 OR ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_max@0 >= 1 AND ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_min@3 <= 1000 OR ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_max@0 >= 1 AND ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_min@3 <= 1000 OR ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_max@0 >= 1 AND ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_min@3 <= 1000 OR ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_max@0 >= 1 AND ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_min@3 <= 1000 OR ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_max@0 >= 1 AND ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_min@3 <= 1000, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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=] + │ AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ RepartitionExec: partitioning=Hash([ps_partkey@0], 6), input_partitions=6, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ 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=, 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 2, output_partitions=8, input_tasks=4 + │ ProjectionExec: expr=[ps_partkey@1 as ps_partkey, ps_availqty@2 as ps_availqty, ps_supplycost@3 as ps_supplycost, s_nationkey@0 as s_nationkey], metrics=[output_rows=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ 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=, 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=, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/supplier/1.parquet, /testdata/tpch/data/supplier/10.parquet, /testdata/tpch/data/supplier/11.parquet], [/testdata/tpch/data/supplier/12.parquet, /testdata/tpch/data/supplier/13.parquet, /testdata/tpch/data/supplier/14.parquet], [/testdata/tpch/data/supplier/15.parquet, /testdata/tpch/data/supplier/16.parquet, /testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet, /testdata/tpch/data/supplier/4.parquet, /testdata/tpch/data/supplier/5.parquet], [/testdata/tpch/data/supplier/6.parquet, /testdata/tpch/data/supplier/7.parquet, /testdata/tpch/data/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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/data/partsupp/1.parquet:.., /testdata/tpch/data/partsupp/10.parquet:.., /testdata/tpch/data/partsupp/11.parquet:..], [/testdata/tpch/data/partsupp/11.parquet:.., /testdata/tpch/data/partsupp/12.parquet:.., /testdata/tpch/data/partsupp/13.parquet:.., /testdata/tpch/data/partsupp/14.parquet:..], [/testdata/tpch/data/partsupp/14.parquet:.., /testdata/tpch/data/partsupp/15.parquet:.., /testdata/tpch/data/partsupp/16.parquet:.., /testdata/tpch/data/partsupp/2.parquet:..], [/testdata/tpch/data/partsupp/2.parquet:.., /testdata/tpch/data/partsupp/3.parquet:.., /testdata/tpch/data/partsupp/4.parquet:..], [/testdata/tpch/data/partsupp/4.parquet:.., /testdata/tpch/data/partsupp/5.parquet:.., /testdata/tpch/data/partsupp/6.parquet:.., /testdata/tpch/data/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ ps_suppkey@1 >= 1 AND ps_suppkey@1 <= 1000 OR ps_suppkey@1 >= 1 AND ps_suppkey@1 <= 1000 OR ps_suppkey@1 >= 1 AND ps_suppkey@1 <= 1000 OR ps_suppkey@1 >= 1 AND ps_suppkey@1 <= 1000 OR ps_suppkey@1 >= 1 AND ps_suppkey@1 <= 1000 OR ps_suppkey@1 >= 1 AND ps_suppkey@1 <= 1000 ], pruning_predicate=ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_max@0 >= 1 AND ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_min@3 <= 1000 OR ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_max@0 >= 1 AND ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_min@3 <= 1000 OR ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_max@0 >= 1 AND ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_min@3 <= 1000 OR ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_max@0 >= 1 AND ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_min@3 <= 1000 OR ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_max@0 >= 1 AND ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_min@3 <= 1000 OR ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_max@0 >= 1 AND ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_min@3 <= 1000, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = GERMANY, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1, required_guarantees=[n_name in (GERMANY)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p2,p3] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = GERMANY, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1, required_guarantees=[n_name in (GERMANY)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = GERMANY, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1, required_guarantees=[n_name in (GERMANY)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = GERMANY, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1, required_guarantees=[n_name in (GERMANY)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = GERMANY, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1, required_guarantees=[n_name in (GERMANY)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p2,p3] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = GERMANY, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1, required_guarantees=[n_name in (GERMANY)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = GERMANY, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1, required_guarantees=[n_name in (GERMANY)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = GERMANY, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1, required_guarantees=[n_name in (GERMANY)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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(()) } #[tokio::test] async fn test_tpch_12() -> Result<(), Box> { - let plan = test_tpch_query(12).await?; + let (plan, analyzed_plan) = test_tpch_query(12).await?; assert_snapshot!(plan, @r#" ┌───── Stage 5 Tasks: t0:[p0] │ SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST] @@ -679,12 +1874,112 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderpriority], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── "#); + assert_snapshot!(analyzed_plan, @r#" + ┌───── Stage 5 Task t0:[p0] + │ SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST], metrics=[output_rows=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 4, output_partitions=24, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t0:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[l_shipmode@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t1:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[l_shipmode@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t2:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[l_shipmode@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[l_shipmode@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([l_shipmode@0], 24), input_partitions=6, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 1, output_partitions=6, n_tasks=3, input_tasks=3 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([l_shipmode@0], 24), input_partitions=6, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 1, output_partitions=6, n_tasks=3, input_tasks=3 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([l_shipmode@0], 24), input_partitions=6, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 1, output_partitions=6, n_tasks=3, input_tasks=3 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 1 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], file_type=parquet, predicate=(l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, pruning_predicate=(l_shipmode_null_count@2 != row_count@3 AND l_shipmode_min@0 <= MAIL AND MAIL <= l_shipmode_max@1 OR l_shipmode_null_count@2 != row_count@3 AND l_shipmode_min@0 <= SHIP AND SHIP <= l_shipmode_max@1) AND l_receiptdate_null_count@5 != row_count@3 AND l_receiptdate_max@4 >= 1994-01-01 AND l_receiptdate_null_count@5 != row_count@3 AND l_receiptdate_min@6 < 1995-01-01, required_guarantees=[l_shipmode in (MAIL, SHIP)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], file_type=parquet, predicate=(l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, pruning_predicate=(l_shipmode_null_count@2 != row_count@3 AND l_shipmode_min@0 <= MAIL AND MAIL <= l_shipmode_max@1 OR l_shipmode_null_count@2 != row_count@3 AND l_shipmode_min@0 <= SHIP AND SHIP <= l_shipmode_max@1) AND l_receiptdate_null_count@5 != row_count@3 AND l_receiptdate_max@4 >= 1994-01-01 AND l_receiptdate_null_count@5 != row_count@3 AND l_receiptdate_min@6 < 1995-01-01, required_guarantees=[l_shipmode in (MAIL, SHIP)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], file_type=parquet, predicate=(l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, pruning_predicate=(l_shipmode_null_count@2 != row_count@3 AND l_shipmode_min@0 <= MAIL AND MAIL <= l_shipmode_max@1 OR l_shipmode_null_count@2 != row_count@3 AND l_shipmode_min@0 <= SHIP AND SHIP <= l_shipmode_max@1) AND l_receiptdate_null_count@5 != row_count@3 AND l_receiptdate_max@4 >= 1994-01-01 AND l_receiptdate_null_count@5 != row_count@3 AND l_receiptdate_min@6 < 1995-01-01, required_guarantees=[l_shipmode in (MAIL, SHIP)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderpriority], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderpriority], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderpriority], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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(()) } #[tokio::test] async fn test_tpch_13() -> Result<(), Box> { - let plan = test_tpch_query(13).await?; + let (plan, analyzed_plan) = test_tpch_query(13).await?; assert_snapshot!(plan, @r" ┌───── Stage 5 Tasks: t0:[p0] │ SortPreservingMergeExec: [custdist@1 DESC, c_count@0 DESC] @@ -722,12 +2017,104 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_comment], file_type=parquet, predicate=o_comment@2 NOT LIKE %special%requests% └────────────────────────────────────────────────── "); + assert_snapshot!(analyzed_plan, @r" + ┌───── Stage 5 Task t0:[p0] + │ SortPreservingMergeExec: [custdist@1 DESC, c_count@0 DESC], metrics=[output_rows=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 4, output_partitions=24, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t0:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[custdist@1 DESC, c_count@0 DESC], preserve_partitioning=[true], metrics=[output_rows=, 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=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(Int64(1))], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t1:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[custdist@1 DESC, c_count@0 DESC], preserve_partitioning=[true], metrics=[output_rows=, 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=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(Int64(1))], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t2:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[custdist@1 DESC, c_count@0 DESC], preserve_partitioning=[true], metrics=[output_rows=, 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=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(Int64(1))], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[custdist@1 DESC, c_count@0 DESC], preserve_partitioning=[true], metrics=[output_rows=, 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=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(Int64(1))], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([c_count@0], 24), input_partitions=6, metrics=[fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(Int64(1))], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count], metrics=[output_rows=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=3, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([c_count@0], 24), input_partitions=6, metrics=[fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(Int64(1))], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count], metrics=[output_rows=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=3, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([c_count@0], 24), input_partitions=6, metrics=[fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(Int64(1))], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count], metrics=[output_rows=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=3, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 2 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([c_custkey@0], 18), input_partitions=6, metrics=[fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ ProjectionExec: expr=[c_custkey@1 as c_custkey, o_orderkey@0 as o_orderkey], metrics=[output_rows=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 1, output_partitions=8, input_tasks=4 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_comment], file_type=parquet, predicate=o_comment@2 NOT LIKE %special%requests%, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p2,p3] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_comment], file_type=parquet, predicate=o_comment@2 NOT LIKE %special%requests%, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_comment], file_type=parquet, predicate=o_comment@2 NOT LIKE %special%requests%, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_comment], file_type=parquet, predicate=o_comment@2 NOT LIKE %special%requests%, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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(()) } #[tokio::test] async fn test_tpch_14() -> Result<(), Box> { - let plan = test_tpch_query(14).await?; + let (plan, analyzed_plan) = test_tpch_query(14).await?; assert_snapshot!(plan, @r#" ┌───── Stage 4 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] @@ -758,12 +2145,96 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01 AND DynamicFilterPhysicalExpr [ true ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1995-09-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1995-10-01, required_guarantees=[] └────────────────────────────────────────────────── "#); + assert_snapshot!(analyzed_plan, @r#" + ┌───── Stage 4 Task 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=, elapsed_compute=] + │ 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=, elapsed_compute=] + │ CoalescePartitionsExec, metrics=[output_rows=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 3, output_partitions=24, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t0:[p0,p1,p2,p3,p4,p5] + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=, elapsed_compute=] + │ 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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 1, output_partitions=6, n_tasks=4, input_tasks=3 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t1:[p0,p1,p2,p3,p4,p5] + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=, elapsed_compute=] + │ 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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 1, output_partitions=6, n_tasks=4, input_tasks=3 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t2:[p0,p1,p2,p3,p4,p5] + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=, elapsed_compute=] + │ 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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 1, output_partitions=6, n_tasks=4, input_tasks=3 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t3:[p0,p1,p2,p3,p4,p5] + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=, elapsed_compute=] + │ 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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 1, output_partitions=6, n_tasks=4, input_tasks=3 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 1 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([p_partkey@0], 24), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_type], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([p_partkey@0], 24), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_type], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([p_partkey@0], 24), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_type], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([l_partkey@0], 24), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01 AND DynamicFilterPhysicalExpr [ true ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1995-09-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1995-10-01, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([l_partkey@0], 24), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01 AND DynamicFilterPhysicalExpr [ true ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1995-09-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1995-10-01, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([l_partkey@0], 24), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01 AND DynamicFilterPhysicalExpr [ true ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1995-09-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1995-10-01, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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(()) } #[tokio::test] async fn test_tpch_15() -> Result<(), Box> { - let plan = test_tpch_query(15).await?; + let (plan, analyzed_plan) = test_tpch_query(15).await?; assert_snapshot!(plan, @r" ┌───── Stage 4 Tasks: t0:[p0] │ SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST] @@ -806,12 +2277,107 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1996-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1996-04-01, required_guarantees=[] └────────────────────────────────────────────────── "); + assert_snapshot!(analyzed_plan, @r" + ┌───── Stage 4 Task t0:[p0] + │ SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST], metrics=[output_rows=, elapsed_compute=] + │ SortExec: expr=[s_suppkey@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ 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=, 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=[max(revenue0.total_revenue)], metrics=[output_rows=, elapsed_compute=] + │ CoalescePartitionsExec, metrics=[output_rows=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 2, output_partitions=24, input_tasks=4 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ 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=, 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=, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/supplier/1.parquet, /testdata/tpch/data/supplier/10.parquet, /testdata/tpch/data/supplier/11.parquet], [/testdata/tpch/data/supplier/12.parquet, /testdata/tpch/data/supplier/13.parquet, /testdata/tpch/data/supplier/14.parquet], [/testdata/tpch/data/supplier/15.parquet, /testdata/tpch/data/supplier/16.parquet, /testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet, /testdata/tpch/data/supplier/4.parquet, /testdata/tpch/data/supplier/5.parquet], [/testdata/tpch/data/supplier/6.parquet, /testdata/tpch/data/supplier/7.parquet, /testdata/tpch/data/supplier/8.parquet], ...]}, projection=[s_suppkey, s_name, s_address, s_phone], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=1, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 2 Task t0:[p0,p1,p2,p3,p4,p5] + │ AggregateExec: mode=Partial, gby=[], aggr=[max(revenue0.total_revenue)], metrics=[output_rows=, elapsed_compute=] + │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue], metrics=[output_rows=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 1, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 2 Task t1:[p0,p1,p2,p3,p4,p5] + │ AggregateExec: mode=Partial, gby=[], aggr=[max(revenue0.total_revenue)], metrics=[output_rows=, elapsed_compute=] + │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue], metrics=[output_rows=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 1, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 2 Task t2:[p0,p1,p2,p3,p4,p5] + │ AggregateExec: mode=Partial, gby=[], aggr=[max(revenue0.total_revenue)], metrics=[output_rows=, elapsed_compute=] + │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue], metrics=[output_rows=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 1, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 2 Task t3:[p0,p1,p2,p3,p4,p5] + │ AggregateExec: mode=Partial, gby=[], aggr=[max(revenue0.total_revenue)], metrics=[output_rows=, elapsed_compute=] + │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue], metrics=[output_rows=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 1, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 1 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([l_suppkey@0], 24), input_partitions=2, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1996-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1996-04-01, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([l_suppkey@0], 24), input_partitions=2, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1996-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1996-04-01, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([l_suppkey@0], 24), input_partitions=2, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1996-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1996-04-01, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([l_suppkey@0], 6), input_partitions=2, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1996-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1996-04-01, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([l_suppkey@0], 6), input_partitions=2, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1996-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1996-04-01, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([l_suppkey@0], 6), input_partitions=2, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1996-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1996-04-01, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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(()) } #[tokio::test] async fn test_tpch_16() -> Result<(), Box> { - let plan = test_tpch_query(16).await?; + let (plan, analyzed_plan) = test_tpch_query(16).await?; assert_snapshot!(plan, @r#" ┌───── Stage 6 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] @@ -858,12 +2424,129 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_brand, p_type, p_size], file_type=parquet, predicate=p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(49), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(14), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(23), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(45), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(19), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(3), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(36), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(9), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), pruning_predicate=p_brand_null_count@2 != row_count@3 AND (p_brand_min@0 != Brand#45 OR Brand#45 != p_brand_max@1) AND p_type_null_count@6 != row_count@3 AND (p_type_min@4 NOT LIKE MEDIUM POLISHED% OR p_type_max@5 NOT LIKE MEDIUM POLISHED%) AND (p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 49 AND 49 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 14 AND 14 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 23 AND 23 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 45 AND 45 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 19 AND 19 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 3 AND 3 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 36 AND 36 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 9 AND 9 <= p_size_max@8), required_guarantees=[p_brand not in (Brand#45), p_size in (14, 19, 23, 3, 36, 45, 49, 9)] └────────────────────────────────────────────────── "#); + assert_snapshot!(analyzed_plan, @r#" + ┌───── Stage 6 Task 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 5, output_partitions=24, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 5 Task t0:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 4, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 5 Task t1:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 4, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 5 Task t2:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 4, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 5 Task t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 4, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2], 24), input_partitions=6, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=3, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2], 24), input_partitions=6, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=3, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2], 24), input_partitions=6, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=3, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 18), input_partitions=6, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(s_suppkey@0, ps_suppkey@0)], metrics=[output_rows=, 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 1, 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], metrics=[output_rows=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 2, output_partitions=8, input_tasks=4 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/partsupp/1.parquet:.., /testdata/tpch/data/partsupp/10.parquet:.., /testdata/tpch/data/partsupp/11.parquet:..], [/testdata/tpch/data/partsupp/11.parquet:.., /testdata/tpch/data/partsupp/12.parquet:.., /testdata/tpch/data/partsupp/13.parquet:.., /testdata/tpch/data/partsupp/14.parquet:..], [/testdata/tpch/data/partsupp/14.parquet:.., /testdata/tpch/data/partsupp/15.parquet:.., /testdata/tpch/data/partsupp/16.parquet:.., /testdata/tpch/data/partsupp/2.parquet:..], [/testdata/tpch/data/partsupp/2.parquet:.., /testdata/tpch/data/partsupp/3.parquet:.., /testdata/tpch/data/partsupp/4.parquet:..], [/testdata/tpch/data/partsupp/4.parquet:.., /testdata/tpch/data/partsupp/5.parquet:.., /testdata/tpch/data/partsupp/6.parquet:.., /testdata/tpch/data/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/supplier/1.parquet, /testdata/tpch/data/supplier/10.parquet, /testdata/tpch/data/supplier/11.parquet], [/testdata/tpch/data/supplier/12.parquet, /testdata/tpch/data/supplier/13.parquet, /testdata/tpch/data/supplier/14.parquet], [/testdata/tpch/data/supplier/15.parquet, /testdata/tpch/data/supplier/16.parquet, /testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet, /testdata/tpch/data/supplier/4.parquet, /testdata/tpch/data/supplier/5.parquet], [/testdata/tpch/data/supplier/6.parquet, /testdata/tpch/data/supplier/7.parquet, /testdata/tpch/data/supplier/8.parquet], ...]}, projection=[s_suppkey, s_comment], file_type=parquet, predicate=s_comment@1 LIKE %Customer%Complaints%, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p2,p3] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/supplier/1.parquet, /testdata/tpch/data/supplier/10.parquet, /testdata/tpch/data/supplier/11.parquet], [/testdata/tpch/data/supplier/12.parquet, /testdata/tpch/data/supplier/13.parquet, /testdata/tpch/data/supplier/14.parquet], [/testdata/tpch/data/supplier/15.parquet, /testdata/tpch/data/supplier/16.parquet, /testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet, /testdata/tpch/data/supplier/4.parquet, /testdata/tpch/data/supplier/5.parquet], [/testdata/tpch/data/supplier/6.parquet, /testdata/tpch/data/supplier/7.parquet, /testdata/tpch/data/supplier/8.parquet], ...]}, projection=[s_suppkey, s_comment], file_type=parquet, predicate=s_comment@1 LIKE %Customer%Complaints%, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/supplier/1.parquet, /testdata/tpch/data/supplier/10.parquet, /testdata/tpch/data/supplier/11.parquet], [/testdata/tpch/data/supplier/12.parquet, /testdata/tpch/data/supplier/13.parquet, /testdata/tpch/data/supplier/14.parquet], [/testdata/tpch/data/supplier/15.parquet, /testdata/tpch/data/supplier/16.parquet, /testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet, /testdata/tpch/data/supplier/4.parquet, /testdata/tpch/data/supplier/5.parquet], [/testdata/tpch/data/supplier/6.parquet, /testdata/tpch/data/supplier/7.parquet, /testdata/tpch/data/supplier/8.parquet], ...]}, projection=[s_suppkey, s_comment], file_type=parquet, predicate=s_comment@1 LIKE %Customer%Complaints%, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/supplier/1.parquet, /testdata/tpch/data/supplier/10.parquet, /testdata/tpch/data/supplier/11.parquet], [/testdata/tpch/data/supplier/12.parquet, /testdata/tpch/data/supplier/13.parquet, /testdata/tpch/data/supplier/14.parquet], [/testdata/tpch/data/supplier/15.parquet, /testdata/tpch/data/supplier/16.parquet, /testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet, /testdata/tpch/data/supplier/4.parquet, /testdata/tpch/data/supplier/5.parquet], [/testdata/tpch/data/supplier/6.parquet, /testdata/tpch/data/supplier/7.parquet, /testdata/tpch/data/supplier/8.parquet], ...]}, projection=[s_suppkey, s_comment], file_type=parquet, predicate=s_comment@1 LIKE %Customer%Complaints%, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(49), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(14), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(23), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(45), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(19), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(3), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(36), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(9), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_brand, p_type, p_size], file_type=parquet, predicate=p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(49), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(14), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(23), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(45), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(19), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(3), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(36), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(9), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), pruning_predicate=p_brand_null_count@2 != row_count@3 AND (p_brand_min@0 != Brand#45 OR Brand#45 != p_brand_max@1) AND p_type_null_count@6 != row_count@3 AND (p_type_min@4 NOT LIKE MEDIUM POLISHED% OR p_type_max@5 NOT LIKE MEDIUM POLISHED%) AND (p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 49 AND 49 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 14 AND 14 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 23 AND 23 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 45 AND 45 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 19 AND 19 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 3 AND 3 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 36 AND 36 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 9 AND 9 <= p_size_max@8), required_guarantees=[p_brand not in (Brand#45), p_size in (14, 19, 23, 3, 36, 45, 49, 9)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p2,p3] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(49), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(14), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(23), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(45), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(19), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(3), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(36), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(9), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_brand, p_type, p_size], file_type=parquet, predicate=p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(49), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(14), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(23), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(45), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(19), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(3), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(36), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(9), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), pruning_predicate=p_brand_null_count@2 != row_count@3 AND (p_brand_min@0 != Brand#45 OR Brand#45 != p_brand_max@1) AND p_type_null_count@6 != row_count@3 AND (p_type_min@4 NOT LIKE MEDIUM POLISHED% OR p_type_max@5 NOT LIKE MEDIUM POLISHED%) AND (p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 49 AND 49 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 14 AND 14 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 23 AND 23 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 45 AND 45 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 19 AND 19 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 3 AND 3 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 36 AND 36 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 9 AND 9 <= p_size_max@8), required_guarantees=[p_brand not in (Brand#45), p_size in (14, 19, 23, 3, 36, 45, 49, 9)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(49), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(14), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(23), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(45), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(19), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(3), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(36), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(9), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_brand, p_type, p_size], file_type=parquet, predicate=p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(49), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(14), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(23), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(45), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(19), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(3), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(36), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(9), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), pruning_predicate=p_brand_null_count@2 != row_count@3 AND (p_brand_min@0 != Brand#45 OR Brand#45 != p_brand_max@1) AND p_type_null_count@6 != row_count@3 AND (p_type_min@4 NOT LIKE MEDIUM POLISHED% OR p_type_max@5 NOT LIKE MEDIUM POLISHED%) AND (p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 49 AND 49 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 14 AND 14 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 23 AND 23 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 45 AND 45 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 19 AND 19 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 3 AND 3 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 36 AND 36 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 9 AND 9 <= p_size_max@8), required_guarantees=[p_brand not in (Brand#45), p_size in (14, 19, 23, 3, 36, 45, 49, 9)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(49), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(14), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(23), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(45), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(19), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(3), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(36), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(9), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_brand, p_type, p_size], file_type=parquet, predicate=p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(49), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(14), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(23), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(45), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(19), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(3), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(36), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(9), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), pruning_predicate=p_brand_null_count@2 != row_count@3 AND (p_brand_min@0 != Brand#45 OR Brand#45 != p_brand_max@1) AND p_type_null_count@6 != row_count@3 AND (p_type_min@4 NOT LIKE MEDIUM POLISHED% OR p_type_max@5 NOT LIKE MEDIUM POLISHED%) AND (p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 49 AND 49 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 14 AND 14 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 23 AND 23 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 45 AND 45 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 19 AND 19 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 3 AND 3 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 36 AND 36 <= p_size_max@8 OR p_size_null_count@9 != row_count@3 AND p_size_min@7 <= 9 AND 9 <= p_size_max@8), required_guarantees=[p_brand not in (Brand#45), p_size in (14, 19, 23, 3, 36, 45, 49, 9)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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(()) } #[tokio::test] async fn test_tpch_17() -> Result<(), Box> { - let plan = test_tpch_query(17).await?; + let (plan, analyzed_plan) = test_tpch_query(17).await?; assert_snapshot!(plan, @r" ┌───── Stage 5 Tasks: t0:[p0] │ ProjectionExec: expr=[CAST(sum(lineitem.l_extendedprice)@0 AS Float64) / 7 as avg_yearly] @@ -904,12 +2587,115 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_quantity], file_type=parquet └────────────────────────────────────────────────── "); + assert_snapshot!(analyzed_plan, @r" + ┌───── Stage 5 Task t0:[p0] + │ ProjectionExec: expr=[CAST(sum(lineitem.l_extendedprice)@0 AS Float64) / 7 as avg_yearly], metrics=[output_rows=, elapsed_compute=] + │ AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice)], metrics=[output_rows=, elapsed_compute=] + │ CoalescePartitionsExec, metrics=[output_rows=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 4, output_partitions=24, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t0:[p0,p1,p2,p3,p4,p5] + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice)], metrics=[output_rows=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=4, input_tasks=1 + │ ProjectionExec: expr=[CAST(0.2 * CAST(avg(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * avg(lineitem.l_quantity), l_partkey@0 as l_partkey], metrics=[output_rows=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t1:[p0,p1,p2,p3,p4,p5] + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice)], metrics=[output_rows=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=4, input_tasks=1 + │ ProjectionExec: expr=[CAST(0.2 * CAST(avg(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * avg(lineitem.l_quantity), l_partkey@0 as l_partkey], metrics=[output_rows=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t2:[p0,p1,p2,p3,p4,p5] + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice)], metrics=[output_rows=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=4, input_tasks=1 + │ ProjectionExec: expr=[CAST(0.2 * CAST(avg(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * avg(lineitem.l_quantity), l_partkey@0 as l_partkey], metrics=[output_rows=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t3:[p0,p1,p2,p3,p4,p5] + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice)], metrics=[output_rows=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 2, output_partitions=6, n_tasks=4, input_tasks=1 + │ ProjectionExec: expr=[CAST(0.2 * CAST(avg(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * avg(lineitem.l_quantity), l_partkey@0 as l_partkey], metrics=[output_rows=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 2 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([p_partkey@2], 24), input_partitions=6, metrics=[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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 1, output_partitions=8, input_tasks=4 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_quantity, l_extendedprice], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_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=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p2,p3] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_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=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_brand, p_container], file_type=parquet, predicate=p_brand@1 = Brand#23 AND p_container@2 = MED BOX, pruning_predicate=p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#23 AND Brand#23 <= p_brand_max@1 AND p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED BOX AND MED BOX <= p_container_max@5, required_guarantees=[p_brand in (Brand#23), p_container in (MED BOX)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_brand, p_container], file_type=parquet, predicate=p_brand@1 = Brand#23 AND p_container@2 = MED BOX, pruning_predicate=p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#23 AND Brand#23 <= p_brand_max@1 AND p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED BOX AND MED BOX <= p_container_max@5, required_guarantees=[p_brand in (Brand#23), p_container in (MED BOX)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([l_partkey@0], 24), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_quantity], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([l_partkey@0], 24), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_quantity], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([l_partkey@0], 24), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_quantity], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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(()) } #[tokio::test] async fn test_tpch_18() -> Result<(), Box> { - let plan = test_tpch_query(18).await?; + let (plan, analyzed_plan) = test_tpch_query(18).await?; assert_snapshot!(plan, @r" ┌───── Stage 9 Tasks: t0:[p0] │ SortPreservingMergeExec: [o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST] @@ -973,12 +2759,174 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_quantity], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── "); + assert_snapshot!(analyzed_plan, @r" + ┌───── Stage 9 Task t0:[p0] + │ SortPreservingMergeExec: [o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST], metrics=[output_rows=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 8, output_partitions=24, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 8 Task t0:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 7, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 8 Task t1:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 7, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 8 Task t2:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 7, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 8 Task t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 7, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 7 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([c_name@0, c_custkey@1, o_orderkey@2, o_orderdate@3, o_totalprice@4], 24), input_partitions=6, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(l_orderkey@0, o_orderkey@2)], metrics=[output_rows=, 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 2, output_partitions=24, input_tasks=4 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 5, output_partitions=6, n_tasks=1, input_tasks=3 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 6, output_partitions=6, n_tasks=1, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 2 Task t0:[p0,p1,p2,p3,p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0], metrics=[output_rows=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 1, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 2 Task t1:[p0,p1,p2,p3,p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0], metrics=[output_rows=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 1, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 2 Task t2:[p0,p1,p2,p3,p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0], metrics=[output_rows=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 1, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 2 Task t3:[p0,p1,p2,p3,p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0], metrics=[output_rows=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 1, output_partitions=6, n_tasks=4, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 1 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 24), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_quantity], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 24), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_quantity], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 24), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_quantity], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([o_orderkey@2], 6), input_partitions=6, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=3, input_tasks=3 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 4, output_partitions=6, n_tasks=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 5 Task t1:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([o_orderkey@2], 6), input_partitions=6, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=3, input_tasks=3 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 4, output_partitions=6, n_tasks=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 5 Task t2:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([o_orderkey@2], 6), input_partitions=6, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=3, input_tasks=3 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 4, output_partitions=6, n_tasks=3, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([c_custkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_name], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([c_custkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_name], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([c_custkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_name], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([o_custkey@1], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([o_custkey@1], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([o_custkey@1], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 6), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_quantity], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 6), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_quantity], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 6), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_quantity], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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(()) } #[tokio::test] async fn test_tpch_19() -> Result<(), Box> { - let plan = test_tpch_query(19).await?; + let (plan, analyzed_plan) = test_tpch_query(19).await?; assert_snapshot!(plan, @r#" ┌───── Stage 2 Tasks: t0:[p0] │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@0 as revenue] @@ -1000,12 +2948,51 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_brand, p_size, p_container], file_type=parquet, predicate=(p_brand@1 = Brand#12 AND p_container@3 IN ([Literal { value: Utf8View("SM CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN ([Literal { value: Utf8View("MED BAG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN ([Literal { value: Utf8View("LG CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 15) AND p_size@2 >= 1, pruning_predicate=(p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#12 AND Brand#12 <= p_brand_max@1 AND (p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM CASE AND SM CASE <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM BOX AND SM BOX <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM PACK AND SM PACK <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM PKG AND SM PKG <= p_container_max@5) AND p_size_null_count@8 != row_count@3 AND p_size_min@7 <= 5 OR p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#23 AND Brand#23 <= p_brand_max@1 AND (p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED BAG AND MED BAG <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED BOX AND MED BOX <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED PKG AND MED PKG <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED PACK AND MED PACK <= p_container_max@5) AND p_size_null_count@8 != row_count@3 AND p_size_min@7 <= 10 OR p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#34 AND Brand#34 <= p_brand_max@1 AND (p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG CASE AND LG CASE <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG BOX AND LG BOX <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG PACK AND LG PACK <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG PKG AND LG PKG <= p_container_max@5) AND p_size_null_count@8 != row_count@3 AND p_size_min@7 <= 15) AND p_size_null_count@8 != row_count@3 AND p_size_max@9 >= 1, required_guarantees=[p_brand in (Brand#12, Brand#23, Brand#34), p_container in (LG BOX, LG CASE, LG PACK, LG PKG, MED BAG, MED BOX, MED PACK, MED PKG, SM BOX, SM CASE, SM PACK, SM PKG)] └────────────────────────────────────────────────── "#); + assert_snapshot!(analyzed_plan, @r#" + ┌───── Stage 2 Task t0:[p0] + │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@0 as revenue], metrics=[output_rows=, elapsed_compute=] + │ AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=, elapsed_compute=] + │ CoalescePartitionsExec, metrics=[output_rows=, elapsed_compute=] + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], filter=p_brand@1 = Brand#12 AND p_container@3 IN ([Literal { value: Utf8View("SM CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN ([Literal { value: Utf8View("MED BAG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN ([Literal { value: Utf8View("LG CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_extendedprice@6, l_discount@7], metrics=[output_rows=, 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 1, output_partitions=8, input_tasks=4 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ 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=, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], file_type=parquet, predicate=(l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2) AND (l_shipmode@5 = AIR OR l_shipmode@5 = AIR REG) AND l_shipinstruct@4 = DELIVER IN PERSON AND DynamicFilterPhysicalExpr [ l_partkey@0 >= 55 AND l_partkey@0 <= 19916 OR l_partkey@0 >= 55 AND l_partkey@0 <= 19916 OR l_partkey@0 >= 55 AND l_partkey@0 <= 19916 OR l_partkey@0 >= 55 AND l_partkey@0 <= 19916 OR l_partkey@0 >= 55 AND l_partkey@0 <= 19916 OR l_partkey@0 >= 55 AND l_partkey@0 <= 19916 ], pruning_predicate=(l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(100),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(1100),15,2 OR l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(1000),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(2000),15,2 OR l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(2000),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(3000),15,2) AND (l_shipmode_null_count@6 != row_count@2 AND l_shipmode_min@4 <= AIR AND AIR <= l_shipmode_max@5 OR l_shipmode_null_count@6 != row_count@2 AND l_shipmode_min@4 <= AIR REG AND AIR REG <= l_shipmode_max@5) AND l_shipinstruct_null_count@9 != row_count@2 AND l_shipinstruct_min@7 <= DELIVER IN PERSON AND DELIVER IN PERSON <= l_shipinstruct_max@8 AND (l_partkey_null_count@11 != row_count@2 AND l_partkey_max@10 >= 55 AND l_partkey_null_count@11 != row_count@2 AND l_partkey_min@12 <= 19916 OR l_partkey_null_count@11 != row_count@2 AND l_partkey_max@10 >= 55 AND l_partkey_null_count@11 != row_count@2 AND l_partkey_min@12 <= 19916 OR l_partkey_null_count@11 != row_count@2 AND l_partkey_max@10 >= 55 AND l_partkey_null_count@11 != row_count@2 AND l_partkey_min@12 <= 19916 OR l_partkey_null_count@11 != row_count@2 AND l_partkey_max@10 >= 55 AND l_partkey_null_count@11 != row_count@2 AND l_partkey_min@12 <= 19916 OR l_partkey_null_count@11 != row_count@2 AND l_partkey_max@10 >= 55 AND l_partkey_null_count@11 != row_count@2 AND l_partkey_min@12 <= 19916 OR l_partkey_null_count@11 != row_count@2 AND l_partkey_max@10 >= 55 AND l_partkey_null_count@11 != row_count@2 AND l_partkey_min@12 <= 19916), required_guarantees=[l_shipinstruct in (DELIVER IN PERSON), l_shipmode in (AIR, AIR REG)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: (p_brand@1 = Brand#12 AND p_container@3 IN ([Literal { value: Utf8View("SM CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN ([Literal { value: Utf8View("MED BAG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN ([Literal { value: Utf8View("LG CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 15) AND p_size@2 >= 1, metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_brand, p_size, p_container], file_type=parquet, predicate=(p_brand@1 = Brand#12 AND p_container@3 IN ([Literal { value: Utf8View("SM CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN ([Literal { value: Utf8View("MED BAG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN ([Literal { value: Utf8View("LG CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 15) AND p_size@2 >= 1, pruning_predicate=(p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#12 AND Brand#12 <= p_brand_max@1 AND (p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM CASE AND SM CASE <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM BOX AND SM BOX <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM PACK AND SM PACK <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM PKG AND SM PKG <= p_container_max@5) AND p_size_null_count@8 != row_count@3 AND p_size_min@7 <= 5 OR p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#23 AND Brand#23 <= p_brand_max@1 AND (p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED BAG AND MED BAG <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED BOX AND MED BOX <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED PKG AND MED PKG <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED PACK AND MED PACK <= p_container_max@5) AND p_size_null_count@8 != row_count@3 AND p_size_min@7 <= 10 OR p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#34 AND Brand#34 <= p_brand_max@1 AND (p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG CASE AND LG CASE <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG BOX AND LG BOX <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG PACK AND LG PACK <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG PKG AND LG PKG <= p_container_max@5) AND p_size_null_count@8 != row_count@3 AND p_size_min@7 <= 15) AND p_size_null_count@8 != row_count@3 AND p_size_max@9 >= 1, required_guarantees=[p_brand in (Brand#12, Brand#23, Brand#34), p_container in (LG BOX, LG CASE, LG PACK, LG PKG, MED BAG, MED BOX, MED PACK, MED PKG, SM BOX, SM CASE, SM PACK, SM PKG)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p2,p3] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: (p_brand@1 = Brand#12 AND p_container@3 IN ([Literal { value: Utf8View("SM CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN ([Literal { value: Utf8View("MED BAG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN ([Literal { value: Utf8View("LG CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 15) AND p_size@2 >= 1, metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_brand, p_size, p_container], file_type=parquet, predicate=(p_brand@1 = Brand#12 AND p_container@3 IN ([Literal { value: Utf8View("SM CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN ([Literal { value: Utf8View("MED BAG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN ([Literal { value: Utf8View("LG CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 15) AND p_size@2 >= 1, pruning_predicate=(p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#12 AND Brand#12 <= p_brand_max@1 AND (p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM CASE AND SM CASE <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM BOX AND SM BOX <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM PACK AND SM PACK <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM PKG AND SM PKG <= p_container_max@5) AND p_size_null_count@8 != row_count@3 AND p_size_min@7 <= 5 OR p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#23 AND Brand#23 <= p_brand_max@1 AND (p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED BAG AND MED BAG <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED BOX AND MED BOX <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED PKG AND MED PKG <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED PACK AND MED PACK <= p_container_max@5) AND p_size_null_count@8 != row_count@3 AND p_size_min@7 <= 10 OR p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#34 AND Brand#34 <= p_brand_max@1 AND (p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG CASE AND LG CASE <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG BOX AND LG BOX <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG PACK AND LG PACK <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG PKG AND LG PKG <= p_container_max@5) AND p_size_null_count@8 != row_count@3 AND p_size_min@7 <= 15) AND p_size_null_count@8 != row_count@3 AND p_size_max@9 >= 1, required_guarantees=[p_brand in (Brand#12, Brand#23, Brand#34), p_container in (LG BOX, LG CASE, LG PACK, LG PKG, MED BAG, MED BOX, MED PACK, MED PKG, SM BOX, SM CASE, SM PACK, SM PKG)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: (p_brand@1 = Brand#12 AND p_container@3 IN ([Literal { value: Utf8View("SM CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN ([Literal { value: Utf8View("MED BAG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN ([Literal { value: Utf8View("LG CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 15) AND p_size@2 >= 1, metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_brand, p_size, p_container], file_type=parquet, predicate=(p_brand@1 = Brand#12 AND p_container@3 IN ([Literal { value: Utf8View("SM CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN ([Literal { value: Utf8View("MED BAG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN ([Literal { value: Utf8View("LG CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 15) AND p_size@2 >= 1, pruning_predicate=(p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#12 AND Brand#12 <= p_brand_max@1 AND (p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM CASE AND SM CASE <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM BOX AND SM BOX <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM PACK AND SM PACK <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM PKG AND SM PKG <= p_container_max@5) AND p_size_null_count@8 != row_count@3 AND p_size_min@7 <= 5 OR p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#23 AND Brand#23 <= p_brand_max@1 AND (p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED BAG AND MED BAG <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED BOX AND MED BOX <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED PKG AND MED PKG <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED PACK AND MED PACK <= p_container_max@5) AND p_size_null_count@8 != row_count@3 AND p_size_min@7 <= 10 OR p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#34 AND Brand#34 <= p_brand_max@1 AND (p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG CASE AND LG CASE <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG BOX AND LG BOX <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG PACK AND LG PACK <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG PKG AND LG PKG <= p_container_max@5) AND p_size_null_count@8 != row_count@3 AND p_size_min@7 <= 15) AND p_size_null_count@8 != row_count@3 AND p_size_max@9 >= 1, required_guarantees=[p_brand in (Brand#12, Brand#23, Brand#34), p_container in (LG BOX, LG CASE, LG PACK, LG PKG, MED BAG, MED BOX, MED PACK, MED PKG, SM BOX, SM CASE, SM PACK, SM PKG)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: (p_brand@1 = Brand#12 AND p_container@3 IN ([Literal { value: Utf8View("SM CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN ([Literal { value: Utf8View("MED BAG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN ([Literal { value: Utf8View("LG CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 15) AND p_size@2 >= 1, metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_brand, p_size, p_container], file_type=parquet, predicate=(p_brand@1 = Brand#12 AND p_container@3 IN ([Literal { value: Utf8View("SM CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN ([Literal { value: Utf8View("MED BAG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN ([Literal { value: Utf8View("LG CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 15) AND p_size@2 >= 1, pruning_predicate=(p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#12 AND Brand#12 <= p_brand_max@1 AND (p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM CASE AND SM CASE <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM BOX AND SM BOX <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM PACK AND SM PACK <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= SM PKG AND SM PKG <= p_container_max@5) AND p_size_null_count@8 != row_count@3 AND p_size_min@7 <= 5 OR p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#23 AND Brand#23 <= p_brand_max@1 AND (p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED BAG AND MED BAG <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED BOX AND MED BOX <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED PKG AND MED PKG <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED PACK AND MED PACK <= p_container_max@5) AND p_size_null_count@8 != row_count@3 AND p_size_min@7 <= 10 OR p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#34 AND Brand#34 <= p_brand_max@1 AND (p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG CASE AND LG CASE <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG BOX AND LG BOX <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG PACK AND LG PACK <= p_container_max@5 OR p_container_null_count@6 != row_count@3 AND p_container_min@4 <= LG PKG AND LG PKG <= p_container_max@5) AND p_size_null_count@8 != row_count@3 AND p_size_min@7 <= 15) AND p_size_null_count@8 != row_count@3 AND p_size_max@9 >= 1, required_guarantees=[p_brand in (Brand#12, Brand#23, Brand#34), p_container in (LG BOX, LG CASE, LG PACK, LG PKG, MED BAG, MED BOX, MED PACK, MED PKG, SM BOX, SM CASE, SM PACK, SM PKG)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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(()) } #[tokio::test] async fn test_tpch_20() -> Result<(), Box> { - let plan = test_tpch_query(20).await?; + let (plan, analyzed_plan) = test_tpch_query(20).await?; assert_snapshot!(plan, @r" ┌───── Stage 4 Tasks: t0:[p0] │ SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] @@ -1052,12 +3039,110 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1994-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1995-01-01, required_guarantees=[] └────────────────────────────────────────────────── "); + assert_snapshot!(analyzed_plan, @r" + ┌───── Stage 4 Task t0:[p0] + │ SortPreservingMergeExec: [s_name@0 ASC NULLS LAST], metrics=[output_rows=, elapsed_compute=] + │ SortExec: expr=[s_name@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_name@1, s_address@2], metrics=[output_rows=, 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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ 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=, 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 1, output_partitions=8, input_tasks=4 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/supplier/1.parquet, /testdata/tpch/data/supplier/10.parquet, /testdata/tpch/data/supplier/11.parquet], [/testdata/tpch/data/supplier/12.parquet, /testdata/tpch/data/supplier/13.parquet, /testdata/tpch/data/supplier/14.parquet], [/testdata/tpch/data/supplier/15.parquet, /testdata/tpch/data/supplier/16.parquet, /testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet, /testdata/tpch/data/supplier/4.parquet, /testdata/tpch/data/supplier/5.parquet], [/testdata/tpch/data/supplier/6.parquet, /testdata/tpch/data/supplier/7.parquet, /testdata/tpch/data/supplier/8.parquet], ...]}, projection=[s_suppkey, s_name, s_address, s_nationkey], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ s_nationkey@3 >= 3 AND s_nationkey@3 <= 3 OR s_nationkey@3 >= 3 AND s_nationkey@3 <= 3 OR s_nationkey@3 >= 3 AND s_nationkey@3 <= 3 OR s_nationkey@3 >= 3 AND s_nationkey@3 <= 3 OR s_nationkey@3 >= 3 AND s_nationkey@3 <= 3 OR s_nationkey@3 >= 3 AND s_nationkey@3 <= 3 ], pruning_predicate=s_nationkey_null_count@1 != row_count@2 AND s_nationkey_max@0 >= 3 AND s_nationkey_null_count@1 != row_count@2 AND s_nationkey_min@3 <= 3 OR s_nationkey_null_count@1 != row_count@2 AND s_nationkey_max@0 >= 3 AND s_nationkey_null_count@1 != row_count@2 AND s_nationkey_min@3 <= 3 OR s_nationkey_null_count@1 != row_count@2 AND s_nationkey_max@0 >= 3 AND s_nationkey_null_count@1 != row_count@2 AND s_nationkey_min@3 <= 3 OR s_nationkey_null_count@1 != row_count@2 AND s_nationkey_max@0 >= 3 AND s_nationkey_null_count@1 != row_count@2 AND s_nationkey_min@3 <= 3 OR s_nationkey_null_count@1 != row_count@2 AND s_nationkey_max@0 >= 3 AND s_nationkey_null_count@1 != row_count@2 AND s_nationkey_min@3 <= 3 OR s_nationkey_null_count@1 != row_count@2 AND s_nationkey_max@0 >= 3 AND s_nationkey_null_count@1 != row_count@2 AND s_nationkey_min@3 <= 3, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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=, elapsed_compute=] + │ 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=, 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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(p_partkey@0, ps_partkey@0)], metrics=[output_rows=, 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 2, output_partitions=8, input_tasks=4 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/partsupp/1.parquet:.., /testdata/tpch/data/partsupp/10.parquet:.., /testdata/tpch/data/partsupp/11.parquet:..], [/testdata/tpch/data/partsupp/11.parquet:.., /testdata/tpch/data/partsupp/12.parquet:.., /testdata/tpch/data/partsupp/13.parquet:.., /testdata/tpch/data/partsupp/14.parquet:..], [/testdata/tpch/data/partsupp/14.parquet:.., /testdata/tpch/data/partsupp/15.parquet:.., /testdata/tpch/data/partsupp/16.parquet:.., /testdata/tpch/data/partsupp/2.parquet:..], [/testdata/tpch/data/partsupp/2.parquet:.., /testdata/tpch/data/partsupp/3.parquet:.., /testdata/tpch/data/partsupp/4.parquet:..], [/testdata/tpch/data/partsupp/4.parquet:.., /testdata/tpch/data/partsupp/5.parquet:.., /testdata/tpch/data/partsupp/6.parquet:.., /testdata/tpch/data/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_availqty], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=1, input_tasks=3 + └────────────────────────────────────────────────── + ┌───── Stage 1 Task t0:[p0,p1] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = CANADA, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= CANADA AND CANADA <= n_name_max@1, required_guarantees=[n_name in (CANADA)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p2,p3] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = CANADA, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= CANADA AND CANADA <= n_name_max@1, required_guarantees=[n_name in (CANADA)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = CANADA, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= CANADA AND CANADA <= n_name_max@1, required_guarantees=[n_name in (CANADA)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = CANADA, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= CANADA AND CANADA <= n_name_max@1, required_guarantees=[n_name in (CANADA)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_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=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p2,p3] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_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=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_name], file_type=parquet, predicate=p_name@1 LIKE forest%, pruning_predicate=p_name_null_count@2 != row_count@3 AND p_name_min@0 <= foresu AND forest <= p_name_max@1, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_name], file_type=parquet, predicate=p_name@1 LIKE forest%, pruning_predicate=p_name_null_count@2 != row_count@3 AND p_name_min@0 <= foresu AND forest <= p_name_max@1, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 6), input_partitions=2, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1994-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1995-01-01, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 6), input_partitions=2, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1994-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1995-01-01, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 6), input_partitions=2, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,p1] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1994-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1995-01-01, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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(()) } #[tokio::test] async fn test_tpch_21() -> Result<(), Box> { - let plan = test_tpch_query(21).await?; + let (plan, analyzed_plan) = test_tpch_query(21).await?; assert_snapshot!(plan, @r" ┌───── Stage 5 Tasks: t0:[p0] │ SortPreservingMergeExec: [numwait@1 DESC, s_name@0 ASC NULLS LAST] @@ -1112,12 +3197,123 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderstatus], file_type=parquet, predicate=o_orderstatus@1 = F, pruning_predicate=o_orderstatus_null_count@2 != row_count@3 AND o_orderstatus_min@0 <= F AND F <= o_orderstatus_max@1, required_guarantees=[o_orderstatus in (F)] └────────────────────────────────────────────────── "); + assert_snapshot!(analyzed_plan, @r" + ┌───── Stage 5 Task t0:[p0] + │ SortPreservingMergeExec: [numwait@1 DESC, s_name@0 ASC NULLS LAST], metrics=[output_rows=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 4, output_partitions=24, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t0:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[numwait@1 DESC, s_name@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[s_name@0 as s_name], aggr=[count(Int64(1))], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t1:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[numwait@1 DESC, s_name@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[s_name@0 as s_name], aggr=[count(Int64(1))], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t2:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[numwait@1 DESC, s_name@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[s_name@0 as s_name], aggr=[count(Int64(1))], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[numwait@1 DESC, s_name@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[s_name@0 as s_name], aggr=[count(Int64(1))], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([s_name@0], 24), input_partitions=6, metrics=[fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[count(Int64(1))], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 1, output_partitions=8, input_tasks=4 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 2, output_partitions=8, input_tasks=4 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/supplier/1.parquet, /testdata/tpch/data/supplier/10.parquet, /testdata/tpch/data/supplier/11.parquet], [/testdata/tpch/data/supplier/12.parquet, /testdata/tpch/data/supplier/13.parquet, /testdata/tpch/data/supplier/14.parquet], [/testdata/tpch/data/supplier/15.parquet, /testdata/tpch/data/supplier/16.parquet, /testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet, /testdata/tpch/data/supplier/4.parquet, /testdata/tpch/data/supplier/5.parquet], [/testdata/tpch/data/supplier/6.parquet, /testdata/tpch/data/supplier/7.parquet, /testdata/tpch/data/supplier/8.parquet], ...]}, projection=[s_suppkey, s_name, s_nationkey], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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=, elapsed_compute=] + │ FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1], metrics=[output_rows=, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@3 > l_commitdate@2 AND DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_suppkey], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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=, elapsed_compute=] + │ FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1], metrics=[output_rows=, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@3 > l_commitdate@2, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = SAUDI ARABIA, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= SAUDI ARABIA AND SAUDI ARABIA <= n_name_max@1, required_guarantees=[n_name in (SAUDI ARABIA)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p2,p3] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = SAUDI ARABIA, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= SAUDI ARABIA AND SAUDI ARABIA <= n_name_max@1, required_guarantees=[n_name in (SAUDI ARABIA)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = SAUDI ARABIA, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= SAUDI ARABIA AND SAUDI ARABIA <= n_name_max@1, required_guarantees=[n_name in (SAUDI ARABIA)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = SAUDI ARABIA, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= SAUDI ARABIA AND SAUDI ARABIA <= n_name_max@1, required_guarantees=[n_name in (SAUDI ARABIA)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderstatus], file_type=parquet, predicate=o_orderstatus@1 = F, pruning_predicate=o_orderstatus_null_count@2 != row_count@3 AND o_orderstatus_min@0 <= F AND F <= o_orderstatus_max@1, required_guarantees=[o_orderstatus in (F)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p2,p3] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderstatus], file_type=parquet, predicate=o_orderstatus@1 = F, pruning_predicate=o_orderstatus_null_count@2 != row_count@3 AND o_orderstatus_min@0 <= F AND F <= o_orderstatus_max@1, required_guarantees=[o_orderstatus in (F)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderstatus], file_type=parquet, predicate=o_orderstatus@1 = F, pruning_predicate=o_orderstatus_null_count@2 != row_count@3 AND o_orderstatus_min@0 <= F AND F <= o_orderstatus_max@1, required_guarantees=[o_orderstatus in (F)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderstatus], file_type=parquet, predicate=o_orderstatus@1 = F, pruning_predicate=o_orderstatus_null_count@2 != row_count@3 AND o_orderstatus_min@0 <= F AND F <= o_orderstatus_max@1, required_guarantees=[o_orderstatus in (F)], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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(()) } #[tokio::test] async fn test_tpch_22() -> Result<(), Box> { - let plan = test_tpch_query(22).await?; + let (plan, analyzed_plan) = test_tpch_query(22).await?; assert_snapshot!(plan, @r#" ┌───── Stage 5 Tasks: t0:[p0] │ SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST] @@ -1158,10 +3354,110 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_phone, c_acctbal], file_type=parquet, predicate=substr(c_phone@1, 1, 2) IN ([Literal { value: Utf8View("13"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("31"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("23"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("29"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("30"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("18"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("17"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) └────────────────────────────────────────────────── "#); + assert_snapshot!(analyzed_plan, @r#" + ┌───── Stage 5 Task t0:[p0] + │ SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST], metrics=[output_rows=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 4, output_partitions=24, input_tasks=4 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t0:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[cntrycode@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t1:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[cntrycode@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t2:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[cntrycode@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 4 Task t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[cntrycode@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=, 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=, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)], metrics=[output_rows=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ NetworkShuffleExec, metrics=[] read_from=Stage 3, output_partitions=6, n_tasks=4, input_tasks=1 + └────────────────────────────────────────────────── + ┌───── Stage 3 Task t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([cntrycode@0], 24), input_partitions=6, metrics=[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=, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ ProjectionExec: expr=[substr(c_phone@0, 1, 2) as cntrycode, c_acctbal@1 as c_acctbal], metrics=[output_rows=, elapsed_compute=] + │ NestedLoopJoinExec: join_type=Inner, filter=CAST(c_acctbal@0 AS Decimal128(19, 6)) > avg(customer.c_acctbal)@1, projection=[c_phone@1, c_acctbal@2], metrics=[output_rows=, 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=, elapsed_compute=] + │ CoalescePartitionsExec, metrics=[output_rows=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 1, output_partitions=8, input_tasks=4 + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, 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=, 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=, elapsed_compute=] + │ NetworkCoalesceExec, metrics=[] read_from=Stage 2, output_partitions=8, input_tasks=4 + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_custkey], file_type=parquet, metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1] + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)], metrics=[output_rows=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN ([Literal { value: Utf8View("13"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("31"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("23"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("29"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("30"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("18"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("17"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), projection=[c_acctbal@1], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_phone, c_acctbal], file_type=parquet, predicate=c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN ([Literal { value: Utf8View("13"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("31"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("23"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("29"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("30"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("18"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("17"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), pruning_predicate=c_acctbal_null_count@1 != row_count@2 AND c_acctbal_max@0 > Some(0),15,2, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p2,p3] + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)], metrics=[output_rows=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN ([Literal { value: Utf8View("13"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("31"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("23"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("29"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("30"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("18"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("17"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), projection=[c_acctbal@1], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_phone, c_acctbal], file_type=parquet, predicate=c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN ([Literal { value: Utf8View("13"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("31"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("23"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("29"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("30"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("18"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("17"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), pruning_predicate=c_acctbal_null_count@1 != row_count@2 AND c_acctbal_max@0 > Some(0),15,2, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p4,p5] + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)], metrics=[output_rows=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN ([Literal { value: Utf8View("13"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("31"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("23"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("29"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("30"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("18"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("17"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), projection=[c_acctbal@1], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_phone, c_acctbal], file_type=parquet, predicate=c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN ([Literal { value: Utf8View("13"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("31"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("23"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("29"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("30"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("18"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("17"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), pruning_predicate=c_acctbal_null_count@1 != row_count@2 AND c_acctbal_max@0 > Some(0),15,2, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t3:[p6,p7] + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)], metrics=[output_rows=, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN ([Literal { value: Utf8View("13"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("31"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("23"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("29"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("30"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("18"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("17"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), projection=[c_acctbal@1], metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_phone, c_acctbal], file_type=parquet, predicate=c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN ([Literal { value: Utf8View("13"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("31"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("23"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("29"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("30"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("18"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("17"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), pruning_predicate=c_acctbal_null_count@1 != row_count@2 AND c_acctbal_max@0 > Some(0),15,2, required_guarantees=[], metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t0:[p0,p1] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: substr(c_phone@1, 1, 2) IN ([Literal { value: Utf8View("13"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("31"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("23"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("29"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("30"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("18"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("17"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t0:[p0,p1,__,__,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_phone, c_acctbal], file_type=parquet, predicate=substr(c_phone@1, 1, 2) IN ([Literal { value: Utf8View("13"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("31"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("23"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("29"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("30"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("18"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("17"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t1:[p2,p3] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: substr(c_phone@1, 1, 2) IN ([Literal { value: Utf8View("13"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("31"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("23"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("29"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("30"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("18"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("17"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t1:[__,__,p0,p1,__,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_phone, c_acctbal], file_type=parquet, predicate=substr(c_phone@1, 1, 2) IN ([Literal { value: Utf8View("13"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("31"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("23"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("29"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("30"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("18"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("17"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t2:[p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: substr(c_phone@1, 1, 2) IN ([Literal { value: Utf8View("13"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("31"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("23"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("29"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("30"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("18"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("17"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t2:[__,__,__,__,p0,__] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_phone, c_acctbal], file_type=parquet, predicate=substr(c_phone@1, 1, 2) IN ([Literal { value: Utf8View("13"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("31"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("23"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("29"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("30"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("18"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("17"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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 Task t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=, elapsed_compute=] + │ FilterExec: substr(c_phone@1, 1, 2) IN ([Literal { value: Utf8View("13"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("31"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("23"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("29"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("30"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("18"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("17"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), metrics=[output_rows=, elapsed_compute=] + │ PartitionIsolatorExec, metrics=[] Task t3:[__,__,__,__,__,p0] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_phone, c_acctbal], file_type=parquet, predicate=substr(c_phone@1, 1, 2) IN ([Literal { value: Utf8View("13"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("31"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("23"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("29"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("30"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("18"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("17"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), metrics=[output_rows=, elapsed_compute=, batches_split=, bytes_scanned=, file_open_errors=, file_scan_errors=, files_ranges_pruned_statistics=, num_predicate_creation_errors=, page_index_rows_matched=, page_index_rows_pruned=, predicate_evaluation_errors=, pushdown_rows_matched=, pushdown_rows_pruned=, row_groups_matched_bloom_filter=, row_groups_matched_statistics=, row_groups_pruned_bloom_filter=, row_groups_pruned_statistics=, 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(()) } - async fn test_tpch_query(query_id: u8) -> Result> { + async fn test_tpch_query(query_id: u8) -> Result<(String, String), Box> { let (ctx, _guard) = start_localhost_context(2, build_state).await; run_tpch_query(ctx, query_id).await } @@ -1181,7 +3477,10 @@ mod tests { // test_non_distributed_consistency runs each TPC-H query twice - once in a distributed manner // and once in a non-distributed manner. For each query, it asserts that the results are identical. - async fn run_tpch_query(ctx_d: SessionContext, query_id: u8) -> Result> { + async fn run_tpch_query( + ctx_d: SessionContext, + query_id: u8, + ) -> Result<(String, String), Box> { ensure_tpch_data(TPCH_SCALE_FACTOR, TPCH_DATA_PARTS).await; let sql = get_test_tpch_query(query_id); ctx_d @@ -1269,7 +3568,9 @@ mod tests { query_id ); let plan_display = displayable(plan_d.as_ref()).indent(true).to_string(); - Ok(plan_display) + + let explain_analyze = explain_analyze(plan_d)?; + Ok((plan_display, explain_analyze)) } pub fn get_test_data_dir() -> std::path::PathBuf {