From da51928b665fc439e7cf1e77af03152e422328df Mon Sep 17 00:00:00 2001 From: Jayant Shrivastava Date: Thu, 9 Oct 2025 16:49:40 -0400 Subject: [PATCH 1/2] implement distributed EXPLAIN ANALYZE This change adds support for displaying a distributed EXPLAIN ANALYZE output with metrics. It updates the TPCH validation tests to assert the EXPLAIN ANALYZE output for each query. I collected the single node results [here](https://github.com/datafusion-contrib/datafusion-distributed/blob/js/full-explain-analyze-rebased-comparison/tests/tpch_validation_test.rs) - using the `output_rows` metric values, we can cross check that the distributed plan metrics are correct. Implemenation notes: - Adds `src/explain.rs` to stores the main entrypoint to rendering the output string - We now re-write the whole plan before rendering using `rewrite_distributed_plan_with_metrics()` which is a new public API that takes an executed `DistributedExec` and wraps each node in the appropriate metrics - A user can call this method on an executed plan and traverse it to collect metrics from particular nodes (This may be hard though because all nodes are wrapped in MetricsWrapperExec...) - Adds a `show_metrics: bool` param to `display_plan_ascii` - Significantly refactors TaskMetricsRewriter -> StageMetricsWriter. See comment. Informs: #123 Other follow up work: - https://github.com/datafusion-contrib/datafusion-distributed/issues/185 - https://github.com/datafusion-contrib/datafusion-distributed/issues/184 - https://github.com/datafusion-contrib/datafusion-distributed/issues/188 - https://github.com/datafusion-contrib/datafusion-distributed/issues/189 - https://github.com/datafusion-contrib/datafusion-distributed/issues/190 --- src/distributed_physical_optimizer_rule.rs | 2 +- src/execution_plans/distributed.rs | 38 +- src/execution_plans/metrics.rs | 4 +- src/flight_service/do_get.rs | 5 +- src/lib.rs | 2 + src/metrics/metrics_collecting_stream.rs | 9 +- src/metrics/mod.rs | 3 +- src/metrics/proto.rs | 12 + src/metrics/task_metrics_collector.rs | 36 +- src/metrics/task_metrics_rewriter.rs | 441 ++++++-- src/protobuf/mod.rs | 1 - src/stage.rs | 53 +- src/test_utils/insta.rs | 128 +++ src/test_utils/metrics.rs | 33 +- tests/custom_extension_codec.rs | 2 +- tests/distributed_aggregation.rs | 4 +- tests/highly_distributed_query.rs | 2 +- tests/introspection.rs | 2 +- tests/stateful_execution_plan.rs | 2 +- tests/tpch_validation_test.rs | 1085 +++++++++++++++++++- 20 files changed, 1670 insertions(+), 194 deletions(-) diff --git a/src/distributed_physical_optimizer_rule.rs b/src/distributed_physical_optimizer_rule.rs index 2cc946a..e71d9a8 100644 --- a/src/distributed_physical_optimizer_rule.rs +++ b/src/distributed_physical_optimizer_rule.rs @@ -710,6 +710,6 @@ mod tests { let df = ctx.sql(query).await?; let physical_plan = df.create_physical_plan().await?; - Ok(display_plan_ascii(physical_plan.as_ref())) + Ok(display_plan_ascii(physical_plan.as_ref(), false)) } } diff --git a/src/execution_plans/distributed.rs b/src/execution_plans/distributed.rs index 9fa4aab..e1be91d 100644 --- a/src/execution_plans/distributed.rs +++ b/src/execution_plans/distributed.rs @@ -4,7 +4,9 @@ use crate::execution_plans::common::require_one_child; use crate::protobuf::DistributedCodec; use crate::stage::{ExecutionTask, Stage}; use datafusion::common::exec_err; +use datafusion::common::internal_datafusion_err; use datafusion::common::tree_node::{Transformed, TreeNode}; +use datafusion::error::DataFusionError; use datafusion::execution::{SendableRecordBatchStream, TaskContext}; use datafusion::physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties}; use datafusion_proto::physical_plan::PhysicalExtensionCodec; @@ -12,6 +14,7 @@ use rand::Rng; use std::any::Any; use std::fmt::Formatter; use std::sync::Arc; +use std::sync::Mutex; use url::Url; /// [ExecutionPlan] that executes the inner plan in distributed mode. @@ -20,14 +23,30 @@ use url::Url; /// channel resolver and assigned to each task in each stage. /// 2. Encodes all the plans in protobuf format so that network boundary nodes can send them /// over the wire. -#[derive(Debug, Clone)] +#[derive(Debug)] pub struct DistributedExec { pub plan: Arc, + pub prepared_plan: Arc>>>, } impl DistributedExec { pub fn new(plan: Arc) -> Self { - Self { plan } + Self { + plan, + prepared_plan: Arc::new(Mutex::new(None)), + } + } + + /// Returns the plan which is lazily prepared on execute() and actually gets executed. + /// It is updated on every call to execute(). Returns an error if .execute() has not been called. + pub(crate) fn prepared_plan(&self) -> Result, DataFusionError> { + self.prepared_plan + .lock() + .map_err(|e| internal_datafusion_err!("Failed to lock prepared plan: {}", e))? + .clone() + .ok_or_else(|| { + internal_datafusion_err!("No prepared plan found. Was execute() called?") + }) } fn prepare_plan( @@ -98,7 +117,8 @@ impl ExecutionPlan for DistributedExec { children: Vec>, ) -> datafusion::common::Result> { Ok(Arc::new(DistributedExec { - plan: require_one_child(children)?, + plan: require_one_child(&children)?, + prepared_plan: self.prepared_plan.clone(), })) } @@ -120,7 +140,15 @@ impl ExecutionPlan for DistributedExec { let channel_resolver = get_distributed_channel_resolver(context.session_config())?; let codec = DistributedCodec::new_combined_with_user(context.session_config()); - let plan = self.prepare_plan(&channel_resolver.get_urls()?, &codec)?; - plan.execute(partition, context) + let prepared = self.prepare_plan(&channel_resolver.get_urls()?, &codec)?; + { + let mut guard = self + .prepared_plan + .lock() + .map_err(|e| internal_datafusion_err!("Failed to lock prepared plan: {}", e))?; + *guard = Some(prepared.clone()); + } + + prepared.execute(partition, context) } } diff --git a/src/execution_plans/metrics.rs b/src/execution_plans/metrics.rs index b424c67..9e0bc60 100644 --- a/src/execution_plans/metrics.rs +++ b/src/execution_plans/metrics.rs @@ -11,7 +11,7 @@ use std::fmt::{Debug, Formatter}; /// A transparent wrapper that delegates all execution to its child but returns custom metrics. This node is invisible during display. /// The structure of a plan tree is closely tied to the [TaskMetricsRewriter]. pub struct MetricsWrapperExec { - inner: Arc, + pub(crate) inner: Arc, /// metrics for this plan node. metrics: MetricsSet, /// children is initially None. When used by the [TaskMetricsRewriter], the children will be updated @@ -53,7 +53,7 @@ impl ExecutionPlan for MetricsWrapperExec { } fn properties(&self) -> &PlanProperties { - unimplemented!("MetricsWrapperExec does not implement properties") + self.inner.properties() } fn children(&self) -> Vec<&Arc> { diff --git a/src/flight_service/do_get.rs b/src/flight_service/do_get.rs index da894f2..814f02c 100644 --- a/src/flight_service/do_get.rs +++ b/src/flight_service/do_get.rs @@ -143,8 +143,9 @@ impl ArrowFlightEndpoint { let stream = map_last_stream(stream, move |last| { if num_partitions_remaining.fetch_sub(1, Ordering::SeqCst) == 1 { task_data_entries.remove(key.clone()); + return last.and_then(|el| collect_and_create_metrics_flight_data(key, plan, el)); } - last.and_then(|el| collect_and_create_metrics_flight_data(key, plan, el)) + last }); Ok(Response::new(Box::pin(stream.map_err(|err| match err { @@ -164,7 +165,7 @@ fn collect_and_create_metrics_flight_data( plan: Arc, incoming: FlightData, ) -> Result { - // Get the metrics for the task executed on this worker. Separately, collect metrics for child tasks. + // Get the metrics for the task executed on this worker + child tasks. let mut result = TaskMetricsCollector::new() .collect(plan) .map_err(|err| FlightError::ProtocolError(err.to_string()))?; diff --git a/src/lib.rs b/src/lib.rs index 5d0c492..21bed14 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -27,6 +27,8 @@ pub use flight_service::{ DistributedSessionBuilderContext, MappedDistributedSessionBuilder, MappedDistributedSessionBuilderExt, }; +pub use metrics::rewrite_distributed_plan_with_metrics; pub use stage::{ DistributedTaskContext, ExecutionTask, Stage, display_plan_ascii, display_plan_graphviz, + explain_analyze, }; diff --git a/src/metrics/metrics_collecting_stream.rs b/src/metrics/metrics_collecting_stream.rs index 7e2e4e6..a60ba25 100644 --- a/src/metrics/metrics_collecting_stream.rs +++ b/src/metrics/metrics_collecting_stream.rs @@ -155,7 +155,10 @@ mod tests { tasks: vec![TaskMetrics { stage_key: Some(stage_key.clone()), // use the task number to seed the test metrics set for convenience - metrics: vec![make_test_metrics_set_proto_from_seed(stage_key.task_number)], + metrics: vec![make_test_metrics_set_proto_from_seed( + stage_key.task_number, + 4, + )], }], })), }) @@ -200,7 +203,7 @@ mod tests { assert_eq!(collected_metrics.len(), 1); assert_eq!( collected_metrics[0], - make_test_metrics_set_proto_from_seed(stage_key.task_number) + make_test_metrics_set_proto_from_seed(stage_key.task_number, 4) ); } } @@ -209,7 +212,7 @@ mod tests { async fn test_metrics_collecting_stream_error_missing_stage_key() { let task_metrics_with_no_stage_key = TaskMetrics { stage_key: None, - metrics: vec![make_test_metrics_set_proto_from_seed(1)], + metrics: vec![make_test_metrics_set_proto_from_seed(1, 4)], }; let invalid_app_metadata = FlightAppMetadata { diff --git a/src/metrics/mod.rs b/src/metrics/mod.rs index b82d8c0..a9a6e7f 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 use task_metrics_rewriter::rewrite_distributed_plan_with_metrics; diff --git a/src/metrics/proto.rs b/src/metrics/proto.rs index ad95438..90b3b5f 100644 --- a/src/metrics/proto.rs +++ b/src/metrics/proto.rs @@ -26,6 +26,18 @@ pub struct MetricsSetProto { pub metrics: Vec, } +impl MetricsSetProto { + pub fn new() -> Self { + Self { + metrics: Vec::new(), + } + } + + pub fn push(&mut self, metric: MetricProto) { + self.metrics.push(metric) + } +} + /// MetricValueProto is a protobuf mirror of the [datafusion::physical_plan::metrics::MetricValue] enum. #[derive(Clone, PartialEq, Eq, ::prost::Oneof)] pub enum MetricValueProto { diff --git a/src/metrics/task_metrics_collector.rs b/src/metrics/task_metrics_collector.rs index bd10d96..e6d5f54 100644 --- a/src/metrics/task_metrics_collector.rs +++ b/src/metrics/task_metrics_collector.rs @@ -232,25 +232,7 @@ mod tests { ctx } - /// runs a sql query and returns the coordinator StageExec - async fn plan_sql(ctx: &SessionContext, sql: &str) -> DistributedExec { - let df = ctx.sql(sql).await.unwrap(); - let physical_distributed = df.create_physical_plan().await.unwrap(); - - let stage_exec = match physical_distributed - .as_any() - .downcast_ref::() - { - Some(stage_exec) => stage_exec.clone(), - None => panic!( - "expected StageExec from distributed optimization, got: {}", - physical_distributed.name() - ), - }; - stage_exec - } - - async fn execute_plan(stage_exec: &DistributedExec, ctx: &SessionContext) { + async fn execute_plan(stage_exec: Arc, ctx: &SessionContext) { let task_ctx = ctx.task_ctx(); let stream = stage_exec.execute(0, task_ctx).unwrap(); @@ -270,21 +252,27 @@ mod tests { async fn run_metrics_collection_e2e_test(sql: &str) { // Plan and execute the query let ctx = make_test_ctx().await; - let stage_exec = plan_sql(&ctx, sql).await; - execute_plan(&stage_exec, &ctx).await; + let df = ctx.sql(sql).await.unwrap(); + let plan = df.create_physical_plan().await.unwrap(); + execute_plan(plan.clone(), &ctx).await; + + let dist_exec = plan + .as_any() + .downcast_ref::() + .expect("expected DistributedExec"); // Assert to ensure the distributed test case is sufficiently complex. - let (stages, expected_stage_keys) = get_stages_and_stage_keys(&stage_exec); + let (stages, expected_stage_keys) = get_stages_and_stage_keys(dist_exec); assert!( expected_stage_keys.len() > 1, "expected more than 1 stage key in test. the plan was not distributed):\n{}", - DisplayableExecutionPlan::new(&stage_exec).indent(true) + DisplayableExecutionPlan::new(plan.as_ref()).indent(true) ); // Collect metrics for all tasks from the root StageExec. let collector = TaskMetricsCollector::new(); - let result = collector.collect(stage_exec.plan.clone()).unwrap(); + let result = collector.collect(dist_exec.plan.clone()).unwrap(); // Ensure that there's metrics for each node for each task for each stage. for expected_stage_key in expected_stage_keys { diff --git a/src/metrics/task_metrics_rewriter.rs b/src/metrics/task_metrics_rewriter.rs index 529237e..6ddc334 100644 --- a/src/metrics/task_metrics_rewriter.rs +++ b/src/metrics/task_metrics_rewriter.rs @@ -1,115 +1,259 @@ +use crate::NetworkBoundaryExt; +use crate::execution_plans::DistributedExec; use crate::execution_plans::MetricsWrapperExec; -use crate::execution_plans::NetworkCoalesceExec; -use crate::execution_plans::NetworkShuffleExec; +use crate::metrics::MetricsCollectorResult; +use crate::metrics::TaskMetricsCollector; use crate::metrics::proto::MetricsSetProto; use crate::metrics::proto::metrics_set_proto_to_df; +use crate::protobuf::StageKey; +use crate::stage::Stage; +use bytes::Bytes; +use datafusion::common::HashMap; use datafusion::common::tree_node::Transformed; use datafusion::common::tree_node::TreeNode; use datafusion::common::tree_node::TreeNodeRecursion; -use datafusion::common::tree_node::TreeNodeRewriter; use datafusion::error::Result; use datafusion::physical_plan::ExecutionPlan; use datafusion::physical_plan::internal_err; +use datafusion::physical_plan::metrics::MetricsSet; use std::sync::Arc; +use std::vec; -/// TaskMetricsRewriter is used to enrich a task with metrics by re-writing the plan using [MetricsWrapperExec] nodes. +/// Rewrites a distributed plan with metrics. Does nothing if the root node is not a [DistributedExec]. +/// Returns an error if the distributed plan was not executed. +pub fn rewrite_distributed_plan_with_metrics( + plan: Arc, +) -> Result> { + let Some(distributed_exec) = plan.as_any().downcast_ref::() else { + return Ok(plan); + }; + + // Collect metrics from the DistributedExec's prepared plan. + let MetricsCollectorResult { + task_metrics, // Metrics for the DistributedExec plan + input_task_metrics, // Metrics for all child stages / tasks. + } = TaskMetricsCollector::new().collect(distributed_exec.prepared_plan()?)?; + + // Rewrite the DistributedExec's child plan with metrics. + let dist_exec_plan_with_metrics = + rewrite_local_plan_with_metrics(plan.children()[0].clone(), task_metrics)?; + let plan = plan.with_new_children(vec![dist_exec_plan_with_metrics])?; + + let metrics_collection = Arc::new(input_task_metrics); + + let transformed = plan.transform_down(|plan| { + // Transform all stages using NetworkShuffleExec and NetworkCoalesceExec as barriers. + if let Some(network_boundary) = plan.as_network_boundary() { + return match network_boundary.input_stage() { + Some(stage) => { + // This transform is a bit inefficient because we traverse the plan nodes twice + // For now, we are okay with trading off performance for simplicity. + let plan_with_metrics = + stage_metrics_rewriter(stage, metrics_collection.clone())?; + Ok(Transformed::yes(network_boundary.with_input_stage( + Stage::new( + stage.query_id, + stage.num, + plan_with_metrics, + stage.tasks.len(), + ), + )?)) + } + None => { + internal_err!("Expected input stage to be set in network boundary") + } + }; + } + + Ok(Transformed::no(plan)) + })?; + Ok(transformed.data) +} + +/// Rewrites a local plan with metrics, stopping at network boundaries. +/// +/// Example: /// -/// Ex. for a plan with the form -/// AggregateExec -/// └── ProjectionExec +/// AggregateExec [output_rows = 1, elapsed_compute = 100] +/// └── ProjectionExec [output_rows = 2, elapsed_compute = 200] /// └── NetworkShuffleExec /// -/// the task will be rewritten as +/// The result will be: /// -/// MetricsWrapperExec (wrapped: AggregateExec) -/// └── MetricsWrapperExec (wrapped: ProjectionExec) +/// MetricsWrapperExec (wrapped: AggregateExec) [output_rows = 1, elapsed_compute = 100] +/// └── MetricsWrapperExec (wrapped: ProjectionExec) [output_rows = 2, elapsed_compute = 200] /// └── NetworkShuffleExec -/// (Note that the NetworkShuffleExec node is not wrapped) -pub struct TaskMetricsRewriter { - metrics: Vec, - idx: usize, +pub fn rewrite_local_plan_with_metrics( + plan: Arc, + metrics: Vec, +) -> Result> { + let mut idx = 0; + Ok(plan + .transform_down(|node| { + if node.is_network_boundary() { + return Ok(Transformed::new(node, false, TreeNodeRecursion::Jump)); + } + if idx >= metrics.len() { + return internal_err!("not enough metrics provided to rewrite plan"); + } + let node_metrics = metrics[idx].clone(); + idx += 1; + Ok(Transformed::yes(Arc::new(MetricsWrapperExec::new( + node.clone(), + node_metrics, + )))) + })? + .data) } -impl TaskMetricsRewriter { - /// Create a new TaskMetricsRewriter. The provided metrics will be used to enrich the plan. - #[allow(dead_code)] - pub fn new(metrics: Vec) -> Self { - Self { metrics, idx: 0 } - } - - /// enrich_task_with_metrics rewrites the plan by wrapping nodes. If the length of the provided metrics set vec does not - /// match the number of nodes in the plan, an error will be returned. - #[allow(dead_code)] - pub fn enrich_task_with_metrics( - mut self, - plan: Arc, - ) -> Result> { - let transformed = plan.rewrite(&mut self)?; - if self.idx != self.metrics.len() { - return internal_err!( - "too many metrics sets provided to rewrite task: {} metrics sets provided, {} nodes in plan", - self.metrics.len(), - self.idx - ); - } - Ok(transformed.data) - } -} +/// Enriches a stage with metrics from each task by re-writing the plan using +/// [MetricsWrapperExec] nodes. +/// +/// Example: +/// +/// For a stage with 2 tasks: +/// +/// Task 1: +/// AggregateExec [output_rows = 1, elapsed_compute = 100] +/// └── ProjectionExec [output_rows = 2, elapsed_compute = 200] +/// └── NetworkShuffleExec +/// +/// Task 2: +/// AggregateExec [output_rows = 3, elapsed_compute = 300] +/// └── ProjectionExec [output_rows = 4, elapsed_compute = 400] +/// └── NetworkShuffleExec +/// +/// The result will be: +/// +/// MetricsWrapperExec (wrapped: AggregateExec) [output_rows = 1, output_rows = 3, elapsed_compute = 100, elapsed_compute = 300] +/// └── MetricsWrapperExec (wrapped: ProjectionExec) [output_rows = 2, output_rows = 4, elapsed_compute = 200, elapsed_compute = 400] +/// └── NetworkShuffleExec +/// +/// Note: +/// - The NetworkShuffleExec node is not wrapped +/// - Metrics may be aggregated by type (ex. output_rows) automatically by various datafusion utils. +/// +/// TODO(#184): Collect metrics from network nodes +/// TODO(#185): Add labels for each task +pub fn stage_metrics_rewriter( + stage: &Stage, + metrics_collection: Arc>>, +) -> Result> { + let mut node_idx = 0; -impl TreeNodeRewriter for TaskMetricsRewriter { - type Node = Arc; + let plan = stage.plan.decoded()?; - fn f_down(&mut self, plan: Self::Node) -> Result> { - if plan.as_any().is::() { + plan.clone().transform_down(|plan| { + // Stop at network boundaries. + if plan.as_network_boundary().is_some() { return Ok(Transformed::new(plan, false, TreeNodeRecursion::Jump)); } - if plan.as_any().is::() { - return Ok(Transformed::new(plan, false, TreeNodeRecursion::Jump)); - } - if self.idx >= self.metrics.len() { - return internal_err!( - "not enough metrics provided to rewrite task: {} metrics provided", - self.metrics.len() - ); + + // Collect metrics for this node. It should contain metrics from each task. + let mut stage_metrics = MetricsSetProto::new(); + + for idx in 0..stage.tasks.len() { + let stage_key = StageKey { + query_id: Bytes::from(stage.query_id.as_bytes().to_vec()), + stage_id: stage.num as u64, + task_number: idx as u64, + }; + match metrics_collection.get(&stage_key) { + Some(task_metrics) => { + if node_idx >= task_metrics.len() { + return internal_err!( + "not enough metrics provided to rewrite task: {} metrics provided", + task_metrics.len() + ); + } + let node_metrics = task_metrics[node_idx].clone(); + for metric in node_metrics.metrics.iter() { + stage_metrics.push(metric.clone()); + } + } + None => { + return internal_err!( + "not enough metrics provided to rewrite task: missing metrics for task {} in stage {}", + idx, + stage.num + ); + } + } } - let proto_metrics = &self.metrics[self.idx]; + + node_idx += 1; let wrapped_plan_node: Arc = Arc::new(MetricsWrapperExec::new( plan.clone(), - metrics_set_proto_to_df(proto_metrics)?, + metrics_set_proto_to_df(&stage_metrics)?, )); - let result = Transformed::new(wrapped_plan_node, true, TreeNodeRecursion::Continue); - self.idx += 1; - Ok(result) - } + Ok(Transformed::yes(wrapped_plan_node)) + }).map(|v| v.data) } #[cfg(test)] mod tests { - use crate::metrics::proto::MetricsSetProto; - use crate::metrics::proto::df_metrics_set_to_proto; - use crate::metrics::task_metrics_collector::TaskMetricsCollector; - use crate::metrics::task_metrics_rewriter::TaskMetricsRewriter; + use crate::DistributedExec; + use crate::PartitionIsolatorExec; + use crate::metrics::proto::{ + MetricsSetProto, df_metrics_set_to_proto, metrics_set_proto_to_df, + }; + use crate::metrics::rewrite_distributed_plan_with_metrics; + use crate::metrics::task_metrics_rewriter::stage_metrics_rewriter; + use crate::protobuf::StageKey; + use crate::test_utils::in_memory_channel_resolver::InMemoryChannelResolver; use crate::test_utils::metrics::make_test_metrics_set_proto_from_seed; use crate::test_utils::plans::count_plan_nodes; use crate::test_utils::session_context::register_temp_parquet_table; + use crate::{NetworkBoundaryExt, Stage}; + use bytes::Bytes; use datafusion::arrow::array::{Int32Array, StringArray}; use datafusion::arrow::datatypes::{DataType, Field, Schema}; use datafusion::arrow::record_batch::RecordBatch; + use datafusion::common::HashMap; use datafusion::execution::SessionStateBuilder; + + use datafusion::physical_plan::{ExecutionPlan, collect}; + use itertools::Itertools; + use uuid::Uuid; + use datafusion::prelude::SessionConfig; use datafusion::prelude::SessionContext; use std::sync::Arc; + use crate::DistributedExt; + use crate::DistributedPhysicalOptimizerRule; + use crate::metrics::task_metrics_rewriter::MetricsWrapperExec; + use datafusion::physical_plan::metrics::MetricsSet; + + async fn make_test_ctx() -> SessionContext { + make_test_ctx_inner(false).await + } + + async fn make_test_distributed_ctx() -> SessionContext { + make_test_ctx_inner(true).await + } + /// Creates a non-distributed session context and registers two tables: /// - table1 (id: int, name: string) /// - table2 (id: int, name: string, phone: string, balance: float64) - async fn make_test_ctx() -> SessionContext { - let config = SessionConfig::new().with_target_partitions(2); - let state = SessionStateBuilder::new() + async fn make_test_ctx_inner(distributed: bool) -> SessionContext { + let config = SessionConfig::new().with_target_partitions(4); + let mut builder = SessionStateBuilder::new() .with_default_features() - .with_config(config) - .build(); + .with_config(config); + + if distributed { + builder = builder + .with_distributed_channel_resolver(InMemoryChannelResolver::new()) + .with_physical_optimizer_rule(Arc::new( + DistributedPhysicalOptimizerRule::default() + .with_network_coalesce_tasks(2) + .with_network_shuffle_tasks(2), + )) + } + + let state = builder.build(); let ctx = SessionContext::from(state); // Create test data for table1 @@ -172,12 +316,28 @@ mod tests { ctx } + fn make_test_stage(plan: Arc) -> Stage { + Stage::new(Uuid::new_v4(), 2, plan, 4) + } + + fn collect_metrics_from_plan(plan: &Arc, metrics: &mut Vec) { + metrics.extend(plan.metrics()); + for child in plan.children() { + collect_metrics_from_plan(child, metrics); + } + } + + fn metrics_set_eq(a: &MetricsSet, b: &MetricsSet) -> bool { + // Check equality by converting to proto representation. + df_metrics_set_to_proto(a).unwrap() == df_metrics_set_to_proto(b).unwrap() + } + /// Asserts that we successfully re-write the metrics of a plan generated from the provided SQL query. /// Also asserts that the order which metrics are collected from a plan matches the order which /// they are re-written (ie. ensures we don't assign metrics to the wrong nodes) /// /// Only tests single node plans since the [TaskMetricsRewriter] stops on [NetworkBoundary]. - async fn run_metrics_rewriter_test(sql: &str) { + async fn run_stage_metrics_rewriter_test(sql: &str) { // Generate the plan let ctx = make_test_ctx().await; let plan = ctx @@ -188,48 +348,85 @@ mod tests { .await .unwrap(); - // Generate metrics for each plan node. - let expected_metrics = (0..count_plan_nodes(&plan)) - .map(|i| make_test_metrics_set_proto_from_seed(i as u64 + 10)) - .collect::>(); + let stage = make_test_stage(plan.clone()); - // Rewrite the metrics. - let rewriter = TaskMetricsRewriter::new(expected_metrics.clone()); - let rewritten_plan = rewriter.enrich_task_with_metrics(plan.clone()).unwrap(); + let num_metrics_per_task_per_node = 4; - // Collect metrics - let actual_metrics = TaskMetricsCollector::new() - .collect(rewritten_plan) - .unwrap() - .task_metrics; - - // Assert that all the metrics are present and in the same order. - assert_eq!(actual_metrics.len(), expected_metrics.len()); - for (actual_metrics_set, expected_metrics_set) in actual_metrics - .iter() - .map(|m| df_metrics_set_to_proto(m).unwrap()) - .zip(expected_metrics) - { - assert_eq!(actual_metrics_set, expected_metrics_set); + // Generate metrics for each task and store them in the map. + let mut metrics_collection = HashMap::new(); + for task_id in 0..stage.tasks.len() { + let stage_key = StageKey { + query_id: Bytes::from(stage.query_id.as_bytes().to_vec()), + stage_id: stage.num as u64, + task_number: task_id as u64, + }; + let metrics = (0..count_plan_nodes(&plan)) + .map(|node_id| { + make_test_metrics_set_proto_from_seed( + (node_id * task_id) as u64, + num_metrics_per_task_per_node, + ) + }) + .collect::>(); + + metrics_collection.insert(stage_key, metrics); + } + let metrics_collection = Arc::new(metrics_collection); + + // Rewrite the plan. + let rewritten_plan = stage_metrics_rewriter(&stage, metrics_collection.clone()).unwrap(); + + // Collect metrics from the plan. + let mut actual_metrics = vec![]; + collect_metrics_from_plan(&rewritten_plan, &mut actual_metrics); + assert_eq!(actual_metrics.len(), count_plan_nodes(&plan)); + + // Assert that metrics from all tasks are present. + // actual_stage_node_metrics_set contains metrics for all task ex. [output_rows=1, elapsed_compute=1, output_rows=2, elapsed_compute=2...] + for (node_id, actual_stage_node_metrics_set) in actual_metrics.iter().enumerate() { + // actual_task_node_metrics_set contains metrics for one task ex. [output_rows=1, elapsed_compute=1] + for (task_id, actual_task_node_metrics_set) in actual_stage_node_metrics_set + .iter() + .chunks(num_metrics_per_task_per_node) + .into_iter() + .enumerate() + { + let expected_task_node_metrics = metrics_collection + .get(&StageKey { + query_id: Bytes::from(stage.query_id.as_bytes().to_vec()), + stage_id: stage.num as u64, + task_number: task_id as u64, + }) + .unwrap()[node_id] + .clone(); + + let mut actual_metrics_set = MetricsSet::new(); + actual_task_node_metrics_set + .for_each(|metric| actual_metrics_set.push(metric.clone())); + + let expected_metrics_set = + metrics_set_proto_to_df(&expected_task_node_metrics).unwrap(); // Convert to proto to check for equality. + assert!(metrics_set_eq(&actual_metrics_set, &expected_metrics_set)); + } } } #[tokio::test] - async fn test_metrics_rewriter_1() { - run_metrics_rewriter_test( + async fn test_stage_metrics_rewriter_1() { + run_stage_metrics_rewriter_test( "SELECT sum(balance) / 7.0 as avg_yearly from table2 group by name", ) .await; } #[tokio::test] - async fn test_metrics_rewriter_2() { - run_metrics_rewriter_test("SELECT id, COUNT(*) as count FROM table1 WHERE id > 1 GROUP BY id ORDER BY id LIMIT 10").await; + async fn test_stage_metrics_rewriter_2() { + run_stage_metrics_rewriter_test("SELECT id, COUNT(*) as count FROM table1 WHERE id > 1 GROUP BY id ORDER BY id LIMIT 10").await; } #[tokio::test] - async fn test_metrics_rewriter_3() { - run_metrics_rewriter_test( + async fn test_stage_metrics_rewriter_3() { + run_stage_metrics_rewriter_test( "SELECT sum(balance) / 7.0 as avg_yearly FROM table2 WHERE name LIKE 'customer%' @@ -241,4 +438,60 @@ mod tests { ) .await; } + + #[tokio::test] + async fn test_rewrite_unexecuted_distributed_plan_with_metrics_err() { + let ctx = make_test_distributed_ctx().await; + let plan = ctx + .sql("SELECT id, COUNT(*) as count FROM table1 WHERE id > 1 GROUP BY id ORDER BY id LIMIT 10") + .await + .unwrap() + .create_physical_plan() + .await + .unwrap(); + assert!(plan.as_any().is::()); + assert!(rewrite_distributed_plan_with_metrics(plan).is_err()); + } + + // Assert every plan node has at least one metric except partition isolators, network boundary nodes, and the root DistributedExec node. + fn assert_metrics_present_in_plan(plan: &Arc) { + if let Some(metrics) = plan.metrics() { + assert!(metrics.iter().count() > 0); + } else { + let is_partition_isolator = + if let Some(metrics_wrapper) = plan.as_any().downcast_ref::() { + metrics_wrapper + .inner + .as_any() + .downcast_ref::() + .is_some() + } else { + false + }; + assert!( + plan.is_network_boundary() + || is_partition_isolator + || plan.as_any().is::() + ); + } + for child in plan.children() { + assert_metrics_present_in_plan(child); + } + } + + #[tokio::test] + async fn test_executed_distributed_plan_has_metrics() { + let ctx = make_test_distributed_ctx().await; + let plan = ctx + .sql("SELECT id, COUNT(*) as count FROM table1 WHERE id > 1 GROUP BY id ORDER BY id LIMIT 10") + .await + .unwrap() + .create_physical_plan() + .await + .unwrap(); + collect(plan.clone(), ctx.task_ctx()).await.unwrap(); + assert!(plan.as_any().is::()); + let rewritten_plan = rewrite_distributed_plan_with_metrics(plan).unwrap(); + assert_metrics_present_in_plan(&rewritten_plan); + } } diff --git a/src/protobuf/mod.rs b/src/protobuf/mod.rs index 5de1c15..3a61533 100644 --- a/src/protobuf/mod.rs +++ b/src/protobuf/mod.rs @@ -3,7 +3,6 @@ mod distributed_codec; mod errors; mod user_codec; -#[allow(unused_imports)] pub(crate) use app_metadata::{AppMetadata, FlightAppMetadata, MetricsCollection, TaskMetrics}; pub(crate) use distributed_codec::{DistributedCodec, StageKey}; pub(crate) use errors::{ diff --git a/src/stage.rs b/src/stage.rs index e4db799..65bf93e 100644 --- a/src/stage.rs +++ b/src/stage.rs @@ -3,6 +3,7 @@ use crate::{NetworkShuffleExec, PartitionIsolatorExec}; use datafusion::common::plan_err; use datafusion::error::Result; use datafusion::execution::TaskContext; +use datafusion::physical_plan::display::DisplayableExecutionPlan; use datafusion::physical_plan::{ExecutionPlan, ExecutionPlanProperties, displayable}; use itertools::{Either, Itertools}; use std::collections::VecDeque; @@ -11,7 +12,7 @@ use url::Url; use uuid::Uuid; /// A unit of isolation for a portion of a physical execution plan -/// that can be executed independently and across a network boundary. +/// that can be executed independently and across a network boundary. /// It implements [`ExecutionPlan`] and can be executed to produce a /// stream of record batches. /// @@ -39,10 +40,10 @@ use uuid::Uuid; /// ┌──────┴────────┐ /// ┌────┴────┐ ┌────┴────┐ /// │ stage 4 │ │ Stage 5 │ -/// └─────────┘ └─────────┘ +/// └─────────┘ └─────────┘ /// /// ``` -/// +/// /// Then executing Stage 1 will run its plan locally. Stage 1 has two inputs, Stage 2 and Stage 3. We /// know these will execute on remote resources. As such the plan for Stage 1 must contain an /// [`NetworkShuffleExec`] node that will read the results of Stage 2 and Stage 3 and coalese the @@ -53,14 +54,14 @@ use uuid::Uuid; /// Arrow Flight Ticket: /// /// ```text -/// ┌─────────┐ -/// │ Stage 2 │ -/// └────┬────┘ +/// ┌─────────┐ +/// │ Stage 2 │ +/// └────┬────┘ /// │ /// ┌──────┴────────┐ /// ┌────┴────┐ ┌────┴────┐ /// │ Stage 4 │ │ Stage 5 │ -/// └─────────┘ └─────────┘ +/// └─────────┘ └─────────┘ /// /// ``` /// @@ -168,7 +169,9 @@ impl Stage { } use crate::distributed_physical_optimizer_rule::{NetworkBoundary, NetworkBoundaryExt}; +use crate::rewrite_distributed_plan_with_metrics; use bytes::Bytes; +use datafusion::common::DataFusionError; use datafusion::physical_expr::Partitioning; use datafusion_proto::physical_plan::{AsExecutionPlan, PhysicalExtensionCodec}; use datafusion_proto::protobuf::PhysicalPlanNode; @@ -187,16 +190,28 @@ use prost::Message; /// the Stage tree. use std::fmt::Write; +/// explain_analyze renders an [ExecutionPlan] with metrics. +pub fn explain_analyze(executed: Arc) -> Result { + match executed.as_any().downcast_ref::() { + None => Ok(DisplayableExecutionPlan::with_metrics(executed.as_ref()) + .indent(true) + .to_string()), + Some(_) => { + let executed = rewrite_distributed_plan_with_metrics(executed.clone())?; + Ok(display_plan_ascii(executed.as_ref(), true)) + } + } +} + // Unicode box-drawing characters for creating borders and connections. const LTCORNER: &str = "┌"; // Left top corner const LDCORNER: &str = "└"; // Left bottom corner const VERTICAL: &str = "│"; // Vertical line const HORIZONTAL: &str = "─"; // Horizontal line - -pub fn display_plan_ascii(plan: &dyn ExecutionPlan) -> String { +pub fn display_plan_ascii(plan: &dyn ExecutionPlan, show_metrics: bool) -> String { if let Some(plan) = plan.as_any().downcast_ref::() { let mut f = String::new(); - display_ascii(Either::Left(plan), 0, &mut f).unwrap(); + display_ascii(Either::Left(plan), 0, show_metrics, &mut f).unwrap(); f } else { displayable(plan).indent(true).to_string() @@ -206,6 +221,7 @@ pub fn display_plan_ascii(plan: &dyn ExecutionPlan) -> String { fn display_ascii( stage: Either<&DistributedExec, &Stage>, depth: usize, + show_metrics: bool, f: &mut String, ) -> std::fmt::Result { let plan = match stage { @@ -244,7 +260,7 @@ fn display_ascii( } let mut plan_str = String::new(); - display_inner_ascii(plan, 0, &mut plan_str)?; + display_inner_ascii(plan, 0, show_metrics, &mut plan_str)?; let plan_str = plan_str .split('\n') .filter(|v| !v.is_empty()) @@ -259,7 +275,7 @@ fn display_ascii( HORIZONTAL.repeat(50) )?; for input_stage in find_input_stages(plan.as_ref()) { - display_ascii(Either::Right(input_stage), depth + 1, f)?; + display_ascii(Either::Right(input_stage), depth + 1, show_metrics, f)?; } Ok(()) } @@ -267,9 +283,16 @@ fn display_ascii( fn display_inner_ascii( plan: &Arc, indent: usize, + show_metrics: bool, f: &mut String, ) -> std::fmt::Result { - let node_str = displayable(plan.as_ref()).one_line().to_string(); + let node_str = if show_metrics { + DisplayableExecutionPlan::with_metrics(plan.as_ref()) + .one_line() + .to_string() + } else { + displayable(plan.as_ref()).one_line().to_string() + }; writeln!(f, "{} {node_str}", " ".repeat(indent))?; if plan.is_network_boundary() { @@ -277,7 +300,7 @@ fn display_inner_ascii( } for child in plan.children() { - display_inner_ascii(child, indent + 2, f)?; + display_inner_ascii(child, indent + 2, show_metrics, f)?; } Ok(()) } @@ -724,7 +747,7 @@ fn find_input_stages(plan: &dyn ExecutionPlan) -> Vec<&Stage> { result } -fn find_all_stages(plan: &Arc) -> Vec<&Stage> { +pub(crate) fn find_all_stages(plan: &Arc) -> Vec<&Stage> { let mut result = vec![]; if let Some(plan) = plan.as_network_boundary() { if let Some(stage) = plan.input_stage() { diff --git a/src/test_utils/insta.rs b/src/test_utils/insta.rs index 54c9225..5fca0bf 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+", ".."); + // Since metrics are not deterministic, we replace them with in test outputs. + // Note taht we leave metrics like `output_rows` since that is deterministic. + 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/src/test_utils/metrics.rs b/src/test_utils/metrics.rs index 4335cb7..a768379 100644 --- a/src/test_utils/metrics.rs +++ b/src/test_utils/metrics.rs @@ -2,36 +2,41 @@ use crate::metrics::proto::{ElapsedCompute, EndTimestamp, OutputRows, StartTimes use crate::metrics::proto::{MetricProto, MetricValueProto, MetricsSetProto}; /// creates a "distinct" set of metrics from the provided seed -pub fn make_test_metrics_set_proto_from_seed(seed: u64) -> MetricsSetProto { +pub fn make_test_metrics_set_proto_from_seed(seed: u64, num_metrics: usize) -> MetricsSetProto { const TEST_TIMESTAMP: i64 = 1758200400000000000; // 2025-09-18 13:00:00 UTC - MetricsSetProto { - metrics: vec![ - MetricProto { - metric: Some(MetricValueProto::OutputRows(OutputRows { value: seed })), + + let mut result = MetricsSetProto { metrics: vec![] }; + + for i in 0..num_metrics { + let value = seed + i as u64; + result.push(match i % 4 { + 0 => MetricProto { + metric: Some(MetricValueProto::OutputRows(OutputRows { value })), labels: vec![], partition: None, }, - MetricProto { - metric: Some(MetricValueProto::ElapsedCompute(ElapsedCompute { - value: seed, - })), + + 1 => MetricProto { + metric: Some(MetricValueProto::ElapsedCompute(ElapsedCompute { value })), labels: vec![], partition: None, }, - MetricProto { + 2 => MetricProto { metric: Some(MetricValueProto::StartTimestamp(StartTimestamp { - value: Some(TEST_TIMESTAMP + (seed as i64 * 1_000_000_000)), + value: Some(TEST_TIMESTAMP + (value as i64 * 1_000_000_000)), })), labels: vec![], partition: None, }, - MetricProto { + 3 => MetricProto { metric: Some(MetricValueProto::EndTimestamp(EndTimestamp { - value: Some(TEST_TIMESTAMP + ((seed as i64 + 1) * 1_000_000_000)), + value: Some(TEST_TIMESTAMP + (value as i64 * 1_000_000_000)), })), labels: vec![], partition: None, }, - ], + _ => unreachable!(), + }) } + result } diff --git a/tests/custom_extension_codec.rs b/tests/custom_extension_codec.rs index 402674e..41c3569 100644 --- a/tests/custom_extension_codec.rs +++ b/tests/custom_extension_codec.rs @@ -60,7 +60,7 @@ mod tests { let distributed_plan = build_plan(true)?; let distributed_plan = DistributedPhysicalOptimizerRule::distribute_plan(distributed_plan)?; - assert_snapshot!(display_plan_ascii(distributed_plan.as_ref()), @r" + assert_snapshot!(display_plan_ascii(distributed_plan.as_ref(), false), @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortExec: expr=[numbers@0 DESC NULLS LAST], preserve_partitioning=[false] │ RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=10 diff --git a/tests/distributed_aggregation.rs b/tests/distributed_aggregation.rs index b0ea216..0027dc8 100644 --- a/tests/distributed_aggregation.rs +++ b/tests/distributed_aggregation.rs @@ -28,7 +28,7 @@ mod tests { .with_network_shuffle_tasks(2) .optimize(physical.clone(), &Default::default())?; - let physical_distributed_str = display_plan_ascii(physical_distributed.as_ref()); + let physical_distributed_str = display_plan_ascii(physical_distributed.as_ref(), false); assert_snapshot!(physical_str, @r" @@ -113,7 +113,7 @@ mod tests { .with_network_coalesce_tasks(6) .optimize(physical.clone(), &Default::default())?; - let physical_distributed_str = display_plan_ascii(physical_distributed.as_ref()); + let physical_distributed_str = display_plan_ascii(physical_distributed.as_ref(), false); assert_snapshot!(physical_str, @r" diff --git a/tests/highly_distributed_query.rs b/tests/highly_distributed_query.rs index 37a9a8a..422e519 100644 --- a/tests/highly_distributed_query.rs +++ b/tests/highly_distributed_query.rs @@ -36,7 +36,7 @@ mod tests { let physical_distributed = DistributedPhysicalOptimizerRule::distribute_plan(physical_distributed)?; - let physical_distributed_str = display_plan_ascii(physical_distributed.as_ref()); + let physical_distributed_str = display_plan_ascii(physical_distributed.as_ref(), false); assert_snapshot!(physical_str, @"DataSourceExec: file_groups={1 group: [[/testdata/flights-1m.parquet]]}, projection=[FL_DATE, DEP_DELAY, ARR_DELAY, AIR_TIME, DISTANCE, DEP_TIME, ARR_TIME], file_type=parquet", diff --git a/tests/introspection.rs b/tests/introspection.rs index 793506c..ac0c590 100644 --- a/tests/introspection.rs +++ b/tests/introspection.rs @@ -33,7 +33,7 @@ mod tests { .with_network_coalesce_tasks(2) .optimize(physical.clone(), &Default::default())?; - let physical_distributed_str = display_plan_ascii(physical_distributed.as_ref()); + let physical_distributed_str = display_plan_ascii(physical_distributed.as_ref(), false); assert_snapshot!(physical_distributed_str, @r" diff --git a/tests/stateful_execution_plan.rs b/tests/stateful_execution_plan.rs index 22e01bc..8339c36 100644 --- a/tests/stateful_execution_plan.rs +++ b/tests/stateful_execution_plan.rs @@ -64,7 +64,7 @@ mod tests { let distributed_plan = build_plan()?; let distributed_plan = DistributedPhysicalOptimizerRule::distribute_plan(distributed_plan)?; - assert_snapshot!(display_plan_ascii(distributed_plan.as_ref()), @r" + assert_snapshot!(display_plan_ascii(distributed_plan.as_ref(), false), @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortExec: expr=[numbers@0 DESC NULLS LAST], preserve_partitioning=[false] │ RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=10 diff --git a/tests/tpch_validation_test.rs b/tests/tpch_validation_test.rs index 273320d..8d8df67 100644 --- a/tests/tpch_validation_test.rs +++ b/tests/tpch_validation_test.rs @@ -8,7 +8,7 @@ mod tests { use datafusion_distributed::test_utils::tpch; use datafusion_distributed::{ DistributedPhysicalOptimizerRule, DistributedSessionBuilderContext, assert_snapshot, - display_plan_ascii, + display_plan_ascii, explain_analyze, }; use futures::TryStreamExt; use std::error::Error; @@ -25,7 +25,7 @@ mod tests { #[tokio::test] async fn test_tpch_1() -> Result<(), Box> { - let plan = test_tpch_query(1).await?; + let (plan, plan_with_metrics) = test_tpch_query(1).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST] @@ -48,12 +48,35 @@ 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!(plan_with_metrics, @r" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST], metrics=[output_rows=4, elapsed_compute=] + │ [Stage 2] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=4, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] + │ ProjectionExec: expr=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus, sum(lineitem.l_quantity)@2 as sum_qty, sum(lineitem.l_extendedprice)@3 as sum_base_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@4 as sum_disc_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax)@5 as sum_charge, avg(lineitem.l_quantity)@6 as avg_qty, avg(lineitem.l_extendedprice)@7 as avg_price, avg(lineitem.l_discount)@8 as avg_disc, count(Int64(1))@9 as count_order], metrics=[output_rows=4, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(Int64(1))], metrics=[output_rows=4, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=24, elapsed_compute=] + │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 24), input_partitions=2, 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=24, 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=591856, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=591856, elapsed_compute=] + │ FilterExec: l_shipdate@6 <= 1998-09-02, projection=[l_quantity@0, l_extendedprice@1, l_discount@2, l_tax@3, l_returnflag@4, l_linestatus@5], metrics=[output_rows=591856, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] + │ 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=600572, 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, plan_with_metrics) = test_tpch_query(2).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST] @@ -136,12 +159,95 @@ 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!(plan_with_metrics, @r" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ SortPreservingMergeExec: [s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST], metrics=[output_rows=11264, elapsed_compute=] + │ [Stage 7] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=11264, 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=11264, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=11264, elapsed_compute=] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@1), (ps_supplycost@7, min(partsupp.ps_supplycost)@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, n_name@8], metrics=[output_rows=11264, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16128, elapsed_compute=] + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=11985, elapsed_compute=] + │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([p_partkey@0, ps_supplycost@7], 24), input_partitions=6, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16128, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@9)], projection=[p_partkey@1, p_mfgr@2, s_name@3, s_address@4, s_phone@5, s_acctbal@6, s_comment@7, ps_supplycost@8, n_name@9], metrics=[output_rows=16128, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=16, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ ProjectionExec: expr=[p_partkey@2 as p_partkey, p_mfgr@3 as p_mfgr, s_name@4 as s_name, s_address@5 as s_address, s_phone@6 as s_phone, s_acctbal@7 as s_acctbal, s_comment@8 as s_comment, ps_supplycost@9 as ps_supplycost, n_name@0 as n_name, n_regionkey@1 as n_regionkey], metrics=[output_rows=4672, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=4672, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@4)], projection=[n_name@1, n_regionkey@2, p_partkey@3, p_mfgr@4, s_name@5, s_address@6, s_phone@8, s_acctbal@9, s_comment@10, ps_supplycost@11], metrics=[output_rows=4672, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=400, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=400, 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=292, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=292, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@2)], projection=[s_name@1, s_address@2, s_nationkey@3, s_phone@4, s_acctbal@5, s_comment@6, p_partkey@7, p_mfgr@8, ps_supplycost@10], metrics=[output_rows=292, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=1000, 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=292, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_partkey@0, p_mfgr@1, ps_suppkey@3, ps_supplycost@4], metrics=[output_rows=292, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=73, elapsed_compute=] + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=80000, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16, elapsed_compute=] + │ FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0], metrics=[output_rows=16, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ 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=80, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=73, elapsed_compute=] + │ FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@1], metrics=[output_rows=73, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ 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=20000, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 24), input_partitions=6, 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=11985, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)], metrics=[output_rows=11985, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=11985, elapsed_compute=] + │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([ps_partkey@0], 18), input_partitions=6, 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=11985, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=4177920, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@2)], projection=[ps_partkey@1, ps_supplycost@2], metrics=[output_rows=4177920, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=16, elapsed_compute=] + │ [Stage 4] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ ProjectionExec: expr=[ps_partkey@1 as ps_partkey, ps_supplycost@2 as ps_supplycost, n_regionkey@0 as n_regionkey], metrics=[output_rows=1280000, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1280000, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_regionkey@1, ps_partkey@2, ps_supplycost@3], metrics=[output_rows=1280000, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=400, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=400, 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=80000, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=80000, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@1)], projection=[s_nationkey@1, ps_partkey@2, ps_supplycost@4], metrics=[output_rows=80000, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=1000, 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=80000, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16, elapsed_compute=] + │ FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0], metrics=[output_rows=16, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ 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=80, 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, plan_with_metrics) = test_tpch_query(3).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [revenue@1 DESC, o_orderdate@2 ASC NULLS LAST] @@ -178,12 +284,48 @@ 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!(plan_with_metrics, @r" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ SortPreservingMergeExec: [revenue@1 DESC, o_orderdate@2 ASC NULLS LAST], metrics=[output_rows=1216, elapsed_compute=] + │ [Stage 3] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[revenue@1 DESC, o_orderdate@2 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=1216, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] + │ ProjectionExec: expr=[l_orderkey@0 as l_orderkey, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@3 as revenue, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority], metrics=[output_rows=1216, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=1216, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1216, elapsed_compute=] + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([l_orderkey@0, o_orderdate@1, o_shippriority@2], 24), input_partitions=6, 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=1216, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=3321, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, o_shippriority@2, l_orderkey@3, l_extendedprice@4, l_discount@5], metrics=[output_rows=3321, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=15224, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=15224, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4], metrics=[output_rows=15224, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=3111, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=72678, elapsed_compute=] + │ FilterExec: o_orderdate@2 < 1995-03-15, metrics=[output_rows=72678, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=150000, 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=324322, elapsed_compute=] + │ FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2], metrics=[output_rows=324322, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=600572, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=3111, elapsed_compute=] + │ FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0], metrics=[output_rows=3111, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ 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=15000, 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, plan_with_metrics) = test_tpch_query(4).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST] @@ -214,12 +356,43 @@ 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!(plan_with_metrics, @r" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST], metrics=[output_rows=5, elapsed_compute=] + │ [Stage 3] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[o_orderpriority@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=5, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] + │ ProjectionExec: expr=[o_orderpriority@0 as o_orderpriority, count(Int64(1))@1 as order_count], metrics=[output_rows=5, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))], metrics=[output_rows=5, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=30, elapsed_compute=] + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([o_orderpriority@0], 24), input_partitions=6, metrics=[fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))], metrics=[output_rows=30, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5093, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(l_orderkey@0, o_orderkey@0)], projection=[o_orderpriority@1], metrics=[output_rows=5093, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=379809, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5552, elapsed_compute=] + │ FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2], metrics=[output_rows=5552, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=150000, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=379809, elapsed_compute=] + │ FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0], metrics=[output_rows=379809, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ 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=600572, 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, plan_with_metrics) = test_tpch_query(5).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [revenue@1 DESC] @@ -273,12 +446,66 @@ 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!(plan_with_metrics, @r" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ SortPreservingMergeExec: [revenue@1 DESC], metrics=[output_rows=5, elapsed_compute=] + │ [Stage 4] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[revenue@1 DESC], preserve_partitioning=[true], metrics=[output_rows=5, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] + │ ProjectionExec: expr=[n_name@0 as n_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as revenue], metrics=[output_rows=5, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[n_name@0 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=5, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=30, elapsed_compute=] + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([n_name@0], 24), input_partitions=6, 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=30, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=221440, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@3)], projection=[l_extendedprice@1, l_discount@2, n_name@3], metrics=[output_rows=221440, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=16, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, n_name@0 as n_name, n_regionkey@1 as n_regionkey], metrics=[output_rows=59040, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=59040, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_name@1, n_regionkey@2, l_extendedprice@3, l_discount@4], metrics=[output_rows=59040, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=400, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=400, 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=3690, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=3690, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1), (s_nationkey@1, c_nationkey@0)], projection=[s_nationkey@1, l_extendedprice@4, l_discount@5], metrics=[output_rows=3690, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=1000, 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=92293, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@1, l_orderkey@0)], projection=[c_nationkey@0, l_suppkey@3, l_extendedprice@4, l_discount@5], metrics=[output_rows=92293, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=22958, elapsed_compute=] + │ ProjectionExec: expr=[c_nationkey@1 as c_nationkey, o_orderkey@0 as o_orderkey], metrics=[output_rows=22958, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=22958, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_custkey@1, c_custkey@0)], projection=[o_orderkey@0, c_nationkey@3], metrics=[output_rows=22958, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=22958, elapsed_compute=] + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=15000, 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=600572, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16, elapsed_compute=] + │ FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0], metrics=[output_rows=16, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ 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=80, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=22958, 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=22958, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ 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=150000, 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, plan_with_metrics) = test_tpch_query(6).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * lineitem.l_discount)@0 as revenue] @@ -294,12 +521,28 @@ 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!(plan_with_metrics, @r" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * lineitem.l_discount)@0 as revenue], metrics=[output_rows=1, elapsed_compute=] + │ AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)], metrics=[output_rows=1, elapsed_compute=] + │ CoalescePartitionsExec, metrics=[output_rows=8, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)], metrics=[output_rows=8, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=11618, 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=11618, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ 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=600572, 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, plan_with_metrics) = test_tpch_query(7).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST] @@ -375,12 +618,88 @@ 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!(plan_with_metrics, @r" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST], metrics=[output_rows=4, elapsed_compute=] + │ [Stage 8] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=4, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] + │ ProjectionExec: expr=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year, sum(shipping.volume)@3 as revenue], metrics=[output_rows=4, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)], metrics=[output_rows=4, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=24, elapsed_compute=] + │ [Stage 7] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 24), input_partitions=6, 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=24, 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=164608, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=164608, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, c_nationkey@3)], filter=n_name@0 = FRANCE AND n_name@1 = GERMANY OR n_name@0 = GERMANY AND n_name@1 = FRANCE, projection=[n_name@1, l_extendedprice@2, l_discount@3, l_shipdate@4, n_name@6], metrics=[output_rows=164608, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=32, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_shipdate@3 as l_shipdate, c_nationkey@4 as c_nationkey, n_name@0 as n_name], metrics=[output_rows=250096, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=250096, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@0)], projection=[n_name@1, l_extendedprice@3, l_discount@4, l_shipdate@5, c_nationkey@6], metrics=[output_rows=250096, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=32, elapsed_compute=] + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ ProjectionExec: expr=[s_nationkey@1 as s_nationkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, l_shipdate@4 as l_shipdate, c_nationkey@0 as c_nationkey], metrics=[output_rows=182762, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=182762, elapsed_compute=] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@4)], projection=[c_nationkey@1, s_nationkey@2, l_extendedprice@3, l_discount@4, l_shipdate@5], metrics=[output_rows=182762, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=15000, elapsed_compute=] + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=182762, elapsed_compute=] + │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=32, elapsed_compute=] + │ FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE, metrics=[output_rows=32, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ 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=400, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=32, elapsed_compute=] + │ FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY, metrics=[output_rows=32, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ 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=400, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([c_custkey@0], 6), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] + │ 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=15000, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([o_custkey@4], 6), input_partitions=6, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=182762, elapsed_compute=] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@1, o_orderkey@0)], projection=[s_nationkey@0, l_extendedprice@2, l_discount@3, l_shipdate@4, o_custkey@6], metrics=[output_rows=182762, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=182762, elapsed_compute=] + │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=150000, elapsed_compute=] + │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([l_orderkey@1], 18), input_partitions=6, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=182762, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5, l_shipdate@6], metrics=[output_rows=182762, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=1000, 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=182762, elapsed_compute=] + │ FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31, metrics=[output_rows=182762, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=600572, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] + │ 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=150000, 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, plan_with_metrics) = test_tpch_query(8).await?; assert_snapshot!(plan, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [o_year@0 ASC NULLS LAST] @@ -466,12 +785,98 @@ 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!(plan_with_metrics, @r#" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ SortPreservingMergeExec: [o_year@0 ASC NULLS LAST], metrics=[output_rows=2, elapsed_compute=] + │ [Stage 8] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[o_year@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=2, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] + │ ProjectionExec: expr=[o_year@0 as o_year, sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END)@1 / sum(all_nations.volume)@2 as mkt_share], metrics=[output_rows=2, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[o_year@0 as o_year], aggr=[sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)], metrics=[output_rows=2, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=12, elapsed_compute=] + │ [Stage 7] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([o_year@0], 24), input_partitions=6, 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=12, 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=1155072, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1155072, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@3)], projection=[l_extendedprice@1, l_discount@2, o_orderdate@3, n_name@5], metrics=[output_rows=1155072, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=16, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, o_orderdate@3 as o_orderdate, n_regionkey@4 as n_regionkey, n_name@0 as n_name], metrics=[output_rows=365824, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=365824, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@2)], projection=[n_name@1, l_extendedprice@2, l_discount@3, o_orderdate@5, n_regionkey@6], metrics=[output_rows=365824, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=400, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=400, 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=22864, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=22864, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, c_nationkey@4)], projection=[n_regionkey@1, l_extendedprice@2, l_discount@3, s_nationkey@4, o_orderdate@5], metrics=[output_rows=22864, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=400, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=400, 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=1429, elapsed_compute=] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@3, c_custkey@0)], projection=[l_extendedprice@0, l_discount@1, s_nationkey@2, o_orderdate@4, c_nationkey@6], metrics=[output_rows=1429, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1429, elapsed_compute=] + │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=15000, elapsed_compute=] + │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16, elapsed_compute=] + │ FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0], metrics=[output_rows=16, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ 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=80, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([o_custkey@3], 6), input_partitions=6, 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=1429, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1429, elapsed_compute=] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_custkey@1, o_orderdate@2, l_extendedprice@4, l_discount@5, s_nationkey@6], metrics=[output_rows=1429, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=45624, elapsed_compute=] + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=4485, elapsed_compute=] + │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=45624, elapsed_compute=] + │ FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31, metrics=[output_rows=45624, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] + │ 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=150000, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 18), input_partitions=6, 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=4485, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=4485, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5], metrics=[output_rows=4485, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=1000, 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=4485, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_suppkey@3, l_extendedprice@4, l_discount@5], metrics=[output_rows=4485, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=147, elapsed_compute=] + │ [Stage 3] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=600572, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=147, elapsed_compute=] + │ FilterExec: p_type@1 = ECONOMY ANODIZED STEEL, projection=[p_partkey@0], metrics=[output_rows=147, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ 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=20000, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([c_custkey@0], 6), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] + │ 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=15000, 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, plan_with_metrics) = test_tpch_query(9).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [nation@0 ASC NULLS LAST, o_year@1 DESC] @@ -539,12 +944,80 @@ 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!(plan_with_metrics, @r" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ SortPreservingMergeExec: [nation@0 ASC NULLS LAST, o_year@1 DESC], metrics=[output_rows=175, elapsed_compute=] + │ [Stage 7] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[nation@0 ASC NULLS LAST, o_year@1 DESC], preserve_partitioning=[true], metrics=[output_rows=175, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] + │ ProjectionExec: expr=[nation@0 as nation, o_year@1 as o_year, sum(profit.amount)@2 as sum_profit], metrics=[output_rows=175, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)], metrics=[output_rows=175, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1050, elapsed_compute=] + │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 6 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([nation@0, o_year@1], 24), input_partitions=6, 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=1050, 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=514560, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=514560, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@3)], projection=[n_name@1, l_quantity@2, l_extendedprice@3, l_discount@4, ps_supplycost@6, o_orderdate@7], metrics=[output_rows=514560, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=400, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=400, 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=32160, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=32160, elapsed_compute=] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@7], metrics=[output_rows=32160, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=150000, elapsed_compute=] + │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=32160, elapsed_compute=] + │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 6), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] + │ 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=150000, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 6), input_partitions=6, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=32160, elapsed_compute=] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, ps_suppkey@1), (l_partkey@1, ps_partkey@0)], projection=[l_orderkey@0, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@9], metrics=[output_rows=32160, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=32160, elapsed_compute=] + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=80000, elapsed_compute=] + │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([l_suppkey@2, l_partkey@1], 18), input_partitions=6, 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=32160, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=32160, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@2)], projection=[s_nationkey@1, l_orderkey@2, l_partkey@3, l_suppkey@4, l_quantity@5, l_extendedprice@6, l_discount@7], metrics=[output_rows=32160, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=1000, 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=32160, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_partkey@2, l_suppkey@3, l_quantity@4, l_extendedprice@5, l_discount@6], metrics=[output_rows=32160, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=1075, elapsed_compute=] + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=600572, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1075, elapsed_compute=] + │ FilterExec: p_name@1 LIKE %green%, projection=[p_partkey@0], metrics=[output_rows=1075, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ 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=20000, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] + │ 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=80000, 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, plan_with_metrics) = test_tpch_query(10).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [revenue@2 DESC] @@ -585,12 +1058,53 @@ 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!(plan_with_metrics, @r" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ SortPreservingMergeExec: [revenue@2 DESC], metrics=[output_rows=3767, elapsed_compute=] + │ [Stage 3] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[revenue@2 DESC], preserve_partitioning=[true], metrics=[output_rows=3767, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] + │ ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@7 as revenue, c_acctbal@2 as c_acctbal, n_name@4 as n_name, c_address@5 as c_address, c_phone@3 as c_phone, c_comment@6 as c_comment], metrics=[output_rows=3767, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@2 as c_acctbal, c_phone@3 as c_phone, n_name@4 as n_name, c_address@5 as c_address, c_comment@6 as c_comment], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=3767, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=4648, elapsed_compute=] + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([c_custkey@0, c_name@1, c_acctbal@2, c_phone@3, n_name@4, c_address@5, c_comment@6], 24), input_partitions=6, 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=4648, 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=183024, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=183024, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, c_nationkey@3)], projection=[n_name@1, c_custkey@2, c_name@3, c_address@4, c_phone@6, c_acctbal@7, c_comment@8, l_extendedprice@9, l_discount@10], metrics=[output_rows=183024, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=400, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=400, 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=11439, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@7, l_orderkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@9, l_discount@10], metrics=[output_rows=11439, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=5677, elapsed_compute=] + │ ProjectionExec: expr=[c_custkey@1 as c_custkey, c_name@2 as c_name, c_address@3 as c_address, c_nationkey@4 as c_nationkey, c_phone@5 as c_phone, c_acctbal@6 as c_acctbal, c_comment@7 as c_comment, o_orderkey@0 as o_orderkey], metrics=[output_rows=5677, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5677, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_custkey@1, c_custkey@0)], projection=[o_orderkey@0, c_custkey@2, c_name@3, c_address@4, c_nationkey@5, c_phone@6, c_acctbal@7, c_comment@8], metrics=[output_rows=5677, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=5677, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=15000, 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=148301, elapsed_compute=] + │ FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2], metrics=[output_rows=148301, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=600572, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5677, 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=5677, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ 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=150000, 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, plan_with_metrics) = test_tpch_query(11).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [value@1 DESC] @@ -639,12 +1153,61 @@ 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!(plan_with_metrics, @r" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ SortPreservingMergeExec: [value@1 DESC], metrics=[output_rows=2541, elapsed_compute=] + │ SortExec: expr=[value@1 DESC], preserve_partitioning=[true], metrics=[output_rows=2541, 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=2541, 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=2541, 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=1, elapsed_compute=] + │ AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)], metrics=[output_rows=1, elapsed_compute=] + │ CoalescePartitionsExec, metrics=[output_rows=6, elapsed_compute=] + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)], metrics=[output_rows=6, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=64000, 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=64000, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=16, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ ProjectionExec: expr=[ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost, s_nationkey@0 as s_nationkey], metrics=[output_rows=80000, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=80000, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_nationkey@1, ps_availqty@3, ps_supplycost@4], metrics=[output_rows=80000, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=1000, 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=80000, 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=3716, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=3716, 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=3716, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=64000, 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=64000, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=16, elapsed_compute=] + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ ProjectionExec: expr=[ps_partkey@1 as ps_partkey, ps_availqty@2 as ps_availqty, ps_supplycost@3 as ps_supplycost, s_nationkey@0 as s_nationkey], metrics=[output_rows=80000, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=80000, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, ps_suppkey@1)], projection=[s_nationkey@1, ps_partkey@2, ps_availqty@4, ps_supplycost@5], metrics=[output_rows=80000, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=1000, 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=80000, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16, elapsed_compute=] + │ FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0], metrics=[output_rows=16, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ 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=400, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16, elapsed_compute=] + │ FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0], metrics=[output_rows=16, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ 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=400, 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, plan_with_metrics) = test_tpch_query(12).await?; assert_snapshot!(plan, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST] @@ -680,12 +1243,48 @@ 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!(plan_with_metrics, @r#" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST], metrics=[output_rows=2, elapsed_compute=] + │ [Stage 4] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[l_shipmode@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=2, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] + │ ProjectionExec: expr=[l_shipmode@0 as l_shipmode, sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@1 as high_line_count, sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@2 as low_line_count], metrics=[output_rows=2, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)], metrics=[output_rows=2, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=36, elapsed_compute=] + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([l_shipmode@0], 24), input_partitions=6, 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=36, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=3155, elapsed_compute=] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3], metrics=[output_rows=3155, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=3155, elapsed_compute=] + │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=150000, elapsed_compute=] + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=3155, elapsed_compute=] + │ FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, projection=[l_orderkey@0, l_shipmode@4], metrics=[output_rows=3155, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] + │ 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=600572, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([o_orderkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] + │ 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=150000, 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, plan_with_metrics) = test_tpch_query(13).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [custdist@1 DESC, c_count@0 DESC] @@ -723,12 +1322,50 @@ 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!(plan_with_metrics, @r" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ SortPreservingMergeExec: [custdist@1 DESC, c_count@0 DESC], metrics=[output_rows=37, elapsed_compute=] + │ [Stage 4] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[custdist@1 DESC, c_count@0 DESC], preserve_partitioning=[true], metrics=[output_rows=37, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] + │ ProjectionExec: expr=[c_count@0 as c_count, count(Int64(1))@1 as custdist], metrics=[output_rows=37, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(Int64(1))], metrics=[output_rows=37, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=568, elapsed_compute=] + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([c_count@0], 24), input_partitions=6, metrics=[fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(Int64(1))], metrics=[output_rows=568, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count], metrics=[output_rows=15000, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)], metrics=[output_rows=15000, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=15000, elapsed_compute=] + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([c_custkey@0], 18), input_partitions=6, 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=15000, 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=153318, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=153318, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Right, on=[(o_custkey@1, c_custkey@0)], projection=[o_orderkey@0, c_custkey@2], metrics=[output_rows=153318, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=148318, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=15000, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=148318, elapsed_compute=] + │ FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1], metrics=[output_rows=148318, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ 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=150000, 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, plan_with_metrics) = test_tpch_query(14).await?; assert_snapshot!(plan, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[100 * CAST(sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END)@0 AS Float64) / CAST(sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 AS Float64) as promo_revenue] @@ -759,12 +1396,43 @@ 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!(plan_with_metrics, @r#" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ ProjectionExec: expr=[100 * CAST(sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END)@0 AS Float64) / CAST(sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 AS Float64) as promo_revenue], metrics=[output_rows=1, elapsed_compute=] + │ AggregateExec: mode=Final, gby=[], aggr=[sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=1, elapsed_compute=] + │ CoalescePartitionsExec, metrics=[output_rows=24, elapsed_compute=] + │ [Stage 3] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=24, 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=7630, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=7630, elapsed_compute=] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], projection=[p_type@1, l_extendedprice@3, l_discount@4], metrics=[output_rows=7630, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=20000, elapsed_compute=] + │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=7630, elapsed_compute=] + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([p_partkey@0], 24), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] + │ 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=20000, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([l_partkey@0], 24), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=7630, elapsed_compute=] + │ FilterExec: l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01, projection=[l_partkey@0, l_extendedprice@1, l_discount@2], metrics=[output_rows=7630, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] + │ 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=600572, 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, plan_with_metrics) = test_tpch_query(15).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST] @@ -807,12 +1475,55 @@ 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!(plan_with_metrics, @r" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST], metrics=[output_rows=1, elapsed_compute=] + │ SortExec: expr=[s_suppkey@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=1, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1, 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=1, 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=1, elapsed_compute=] + │ CoalescePartitionsExec, metrics=[output_rows=24, elapsed_compute=] + │ [Stage 2] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1000, 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=1000, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=1000, 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=1000, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=1000, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5825, elapsed_compute=] + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + │ AggregateExec: mode=Partial, gby=[], aggr=[max(revenue0.total_revenue)], metrics=[output_rows=24, elapsed_compute=] + │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue], metrics=[output_rows=1000, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=1000, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5825, elapsed_compute=] + │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([l_suppkey@0], 24), input_partitions=2, 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=5825, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=22830, elapsed_compute=] + │ FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2], metrics=[output_rows=22830, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] + │ 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=600572, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([l_suppkey@0], 6), input_partitions=2, 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=5825, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=22830, elapsed_compute=] + │ FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2], metrics=[output_rows=22830, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] + │ 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=600572, 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, plan_with_metrics) = test_tpch_query(16).await?; assert_snapshot!(plan, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST] @@ -859,12 +1570,59 @@ 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!(plan_with_metrics, @r#" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ SortPreservingMergeExec: [supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST], metrics=[output_rows=2762, elapsed_compute=] + │ [Stage 5] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 5 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=2762, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] + │ ProjectionExec: expr=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, count(alias1)@3 as supplier_cnt], metrics=[output_rows=2762, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[count(alias1)], metrics=[output_rows=2762, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=10538, elapsed_compute=] + │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2], 24), input_partitions=6, 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=10538, 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=11632, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=11634, elapsed_compute=] + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 18), input_partitions=6, 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=11634, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=11635, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(s_suppkey@0, ps_suppkey@0)], metrics=[output_rows=11635, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=1, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ ProjectionExec: expr=[ps_suppkey@3 as ps_suppkey, p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], metrics=[output_rows=11644, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=11644, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_brand@1, p_type@2, p_size@3, ps_suppkey@5], metrics=[output_rows=11644, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=2911, elapsed_compute=] + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=80000, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1, elapsed_compute=] + │ FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0], metrics=[output_rows=1, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ 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=1000, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=2911, 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=2911, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ 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=20000, 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, plan_with_metrics) = test_tpch_query(17).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[CAST(sum(lineitem.l_extendedprice)@0 AS Float64) / 7 as avg_yearly] @@ -905,12 +1663,53 @@ 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!(plan_with_metrics, @r" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ ProjectionExec: expr=[CAST(sum(lineitem.l_extendedprice)@0 AS Float64) / 7 as avg_yearly], metrics=[output_rows=1, elapsed_compute=] + │ AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice)], metrics=[output_rows=1, elapsed_compute=] + │ CoalescePartitionsExec, metrics=[output_rows=24, elapsed_compute=] + │ [Stage 4] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice)], metrics=[output_rows=24, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=43, elapsed_compute=] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * avg(lineitem.l_quantity)@1, projection=[l_extendedprice@1], metrics=[output_rows=43, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=555, elapsed_compute=] + │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + │ ProjectionExec: expr=[CAST(0.2 * CAST(avg(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * avg(lineitem.l_quantity), l_partkey@0 as l_partkey], metrics=[output_rows=20000, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)], metrics=[output_rows=20000, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=118823, elapsed_compute=] + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([p_partkey@2], 24), input_partitions=6, 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=555, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=555, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p_partkey@0, l_partkey@0)], projection=[p_partkey@0, l_quantity@2, l_extendedprice@3], metrics=[output_rows=555, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=18, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=600572, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=18, elapsed_compute=] + │ FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0], metrics=[output_rows=18, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ 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=20000, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([l_partkey@0], 24), input_partitions=2, 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=118823, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] + │ 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=600572, 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, plan_with_metrics) = test_tpch_query(18).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST] @@ -974,12 +1773,76 @@ 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!(plan_with_metrics, @r" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ SortPreservingMergeExec: [o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST], metrics=[output_rows=5, elapsed_compute=] + │ [Stage 8] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 8 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=5, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] + │ AggregateExec: mode=FinalPartitioned, gby=[c_name@0 as c_name, c_custkey@1 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@3 as o_orderdate, o_totalprice@4 as o_totalprice], aggr=[sum(lineitem.l_quantity)], metrics=[output_rows=5, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5, elapsed_compute=] + │ [Stage 7] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 7 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([c_name@0, c_custkey@1, o_orderkey@2, o_orderdate@3, o_totalprice@4], 24), input_partitions=6, 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=5, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=35, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(l_orderkey@0, o_orderkey@2)], metrics=[output_rows=35, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=5, elapsed_compute=] + │ [Stage 2] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=600572, elapsed_compute=] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@2, l_orderkey@0)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@3, o_orderdate@4, l_quantity@6], metrics=[output_rows=600572, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=150000, elapsed_compute=] + │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=600572, elapsed_compute=] + │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5, elapsed_compute=] + │ FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0], metrics=[output_rows=5, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)], metrics=[output_rows=150000, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=150000, elapsed_compute=] + │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 24), input_partitions=2, 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=150000, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] + │ 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=600572, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([o_orderkey@2], 6), input_partitions=6, metrics=[fetch_time=, repartition_time=, send_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=150000, elapsed_compute=] + │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@4, o_orderdate@5], metrics=[output_rows=150000, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=15000, elapsed_compute=] + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=150000, elapsed_compute=] + │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([c_custkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] + │ 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=15000, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + │ RepartitionExec: partitioning=Hash([o_custkey@1], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] + │ 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=150000, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([l_orderkey@0], 6), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] + │ 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=600572, 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, plan_with_metrics) = test_tpch_query(19).await?; assert_snapshot!(plan, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@0 as revenue] @@ -1001,12 +1864,34 @@ 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!(plan_with_metrics, @r#" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@0 as revenue], metrics=[output_rows=1, elapsed_compute=] + │ AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=1, elapsed_compute=] + │ CoalescePartitionsExec, metrics=[output_rows=6, elapsed_compute=] + │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=6, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=10, 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=10, 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=42, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=12635, 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=12635, 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=600572, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=42, 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=42, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ 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=20000, 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, plan_with_metrics) = test_tpch_query(20).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] @@ -1053,12 +1938,59 @@ 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!(plan_with_metrics, @r" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ SortPreservingMergeExec: [s_name@0 ASC NULLS LAST], metrics=[output_rows=144, elapsed_compute=] + │ SortExec: expr=[s_name@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=144, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=144, 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=144, 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=592, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=592, 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=592, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=16, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ 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=1000, 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=521, 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=521, 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=760, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=760, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(p_partkey@0, ps_partkey@0)], metrics=[output_rows=760, 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=190, elapsed_compute=] + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ 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=80000, 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=54539, 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=54539, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=83515, elapsed_compute=] + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16, elapsed_compute=] + │ FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0], metrics=[output_rows=16, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ 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=400, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=190, elapsed_compute=] + │ FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0], metrics=[output_rows=190, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ 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=20000, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + │ RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 6), input_partitions=2, 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=83515, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=92040, elapsed_compute=] + │ FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2], metrics=[output_rows=92040, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] + │ 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=600572, 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, plan_with_metrics) = test_tpch_query(21).await?; assert_snapshot!(plan, @r" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [numwait@1 DESC, s_name@0 ASC NULLS LAST] @@ -1113,12 +2045,66 @@ 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!(plan_with_metrics, @r" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ SortPreservingMergeExec: [numwait@1 DESC, s_name@0 ASC NULLS LAST], metrics=[output_rows=47, elapsed_compute=] + │ [Stage 4] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[numwait@1 DESC, s_name@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=47, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] + │ ProjectionExec: expr=[s_name@0 as s_name, count(Int64(1))@1 as numwait], metrics=[output_rows=47, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[s_name@0 as s_name], aggr=[count(Int64(1))], metrics=[output_rows=47, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=47, elapsed_compute=] + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([s_name@0], 24), input_partitions=6, metrics=[fetch_time=, repartition_time=, send_time=] + │ AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[count(Int64(1))], metrics=[output_rows=47, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=7440, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, projection=[s_name@0], metrics=[output_rows=7440, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=132720, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=132720, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, metrics=[output_rows=132720, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=137440, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=137440, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(n_nationkey@0, s_nationkey@1)], projection=[s_name@1, l_orderkey@3, l_suppkey@4], metrics=[output_rows=137440, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=16, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=182902, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(o_orderkey@0, l_orderkey@2)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4], metrics=[output_rows=182902, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=72884, elapsed_compute=] + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=379809, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4], metrics=[output_rows=379809, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=1000, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=1000, 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=379809, elapsed_compute=] + │ FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1], metrics=[output_rows=379809, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=600572, 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=600572, 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=379809, elapsed_compute=] + │ FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1], metrics=[output_rows=379809, elapsed_compute=] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=600572, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16, elapsed_compute=] + │ FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0], metrics=[output_rows=16, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ 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=400, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=72884, elapsed_compute=] + │ FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0], metrics=[output_rows=72884, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ 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=150000, 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, plan_with_metrics) = test_tpch_query(22).await?; assert_snapshot!(plan, @r#" ┌───── DistributedExec ── Tasks: t0:[p0] │ SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST] @@ -1159,10 +2145,51 @@ 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!(plan_with_metrics, @r#" + ┌───── DistributedExec ── Tasks: t0:[p0] + │ SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST], metrics=[output_rows=7, elapsed_compute=] + │ [Stage 4] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + │ SortExec: expr=[cntrycode@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=7, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] + │ ProjectionExec: expr=[cntrycode@0 as cntrycode, count(Int64(1))@1 as numcust, sum(custsale.c_acctbal)@2 as totacctbal], metrics=[output_rows=7, elapsed_compute=] + │ AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)], metrics=[output_rows=7, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=7, elapsed_compute=] + │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] + └────────────────────────────────────────────────── + ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + │ RepartitionExec: partitioning=Hash([cntrycode@0], 24), input_partitions=6, 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=7, 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=641, 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=641, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)], metrics=[output_rows=1, elapsed_compute=] + │ CoalescePartitionsExec, metrics=[output_rows=8, elapsed_compute=] + │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1360, elapsed_compute=] + │ HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)], projection=[c_phone@1, c_acctbal@2], metrics=[output_rows=1360, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] + │ CoalescePartitionsExec, metrics=[output_rows=4115, elapsed_compute=] + │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] + │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=150000, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + │ AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)], metrics=[output_rows=8, elapsed_compute=] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=3741, 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=3741, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ 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=15000, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=4115, 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=4115, elapsed_compute=] + │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] + │ 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=15000, 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 } @@ -1182,7 +2209,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 +2299,10 @@ mod tests { "Query {} results differ between executions", query_id ); - Ok(display_plan_ascii(plan_d.as_ref())) + + let plan_display = display_plan_ascii(plan_d.as_ref(), false); + let explain_analyze = explain_analyze(plan_d.clone())?; + Ok((plan_display, explain_analyze)) } pub fn get_test_data_dir() -> std::path::PathBuf { From e5059cf0a8d4b115e7d7d6033a611a4c7b187a52 Mon Sep 17 00:00:00 2001 From: Jayant Shrivastava Date: Fri, 17 Oct 2025 06:17:30 -0400 Subject: [PATCH 2/2] display: use range to represent partitions in a task (#194) This change makes the explain output more concise by changing the stage formatting from ``` Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] ``` to ``` Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] ``` Closes: #192 --- src/distributed_physical_optimizer_rule.rs | 20 +- src/stage.rs | 11 +- tests/custom_extension_codec.rs | 4 +- tests/distributed_aggregation.rs | 6 +- tests/stateful_execution_plan.rs | 4 +- tests/tpch_validation_test.rs | 364 ++++++++++----------- 6 files changed, 206 insertions(+), 203 deletions(-) diff --git a/src/distributed_physical_optimizer_rule.rs b/src/distributed_physical_optimizer_rule.rs index e71d9a8..a5ff057 100644 --- a/src/distributed_physical_optimizer_rule.rs +++ b/src/distributed_physical_optimizer_rule.rs @@ -498,14 +498,14 @@ mod tests { │ SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3] t1:[p0,p1,p2,p3] + ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] │ SortExec: expr=[count(*)@0 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday, count(Int64(1))@1 as count(Int64(1))] │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7] t1:[p0,p1,p2,p3,p4,p5,p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] │ RepartitionExec: partitioning=Hash([RainToday@0], 8), input_partitions=4 │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] @@ -526,14 +526,14 @@ mod tests { │ SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3] t1:[p0,p1,p2,p3] + ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] │ SortExec: expr=[count(*)@0 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday, count(Int64(1))@1 as count(Int64(1))] │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7] t1:[p0,p1,p2,p3,p4,p5,p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] │ RepartitionExec: partitioning=Hash([RainToday@0], 8), input_partitions=4 │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] @@ -598,13 +598,13 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 3] => NetworkShuffleExec: output_partitions=4, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3] t1:[p0,p1,p2,p3] + ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] │ ProjectionExec: expr=[avg(weather.MinTemp)@1 as MinTemp, RainTomorrow@0 as RainTomorrow] │ AggregateExec: mode=FinalPartitioned, gby=[RainTomorrow@0 as RainTomorrow], aggr=[avg(weather.MinTemp)] │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7] t1:[p0,p1,p2,p3,p4,p5,p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] │ RepartitionExec: partitioning=Hash([RainTomorrow@0], 8), input_partitions=4 │ AggregateExec: mode=Partial, gby=[RainTomorrow@1 as RainTomorrow], aggr=[avg(weather.MinTemp)] │ CoalesceBatchesExec: target_batch_size=8192 @@ -613,7 +613,7 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__] t1:[__,__,p0] │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday, RainTomorrow], file_type=parquet, predicate=RainToday@1 = yes, pruning_predicate=RainToday_null_count@2 != row_count@3 AND RainToday_min@0 <= yes AND yes <= RainToday_max@1, required_guarantees=[RainToday in (yes)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3] t1:[p0,p1,p2,p3] + ┌───── Stage 3 ── Tasks: t0:[p0..p3] t1:[p0..p3] │ RepartitionExec: partitioning=Hash([RainTomorrow@0], 4), input_partitions=4 │ AggregateExec: mode=Partial, gby=[RainTomorrow@1 as RainTomorrow], aggr=[avg(weather.MaxTemp)] │ CoalesceBatchesExec: target_batch_size=8192 @@ -634,7 +634,7 @@ mod tests { │ SortPreservingMergeExec: [MinTemp@0 DESC] │ [Stage 1] => NetworkCoalesceExec: output_partitions=4, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] │ SortExec: expr=[MinTemp@0 DESC], preserve_partitioning=[true] │ PartitionIsolatorExec: t0:[p0,p1,__] t1:[__,__,p0] │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, MaxTemp, Rainfall, Evaporation, Sunshine, WindGustDir, WindGustSpeed, WindDir9am, WindDir3pm, WindSpeed9am, WindSpeed3pm, Humidity9am, Humidity3pm, Pressure9am, Pressure3pm, Cloud9am, Cloud3pm, Temp9am, Temp3pm, RainToday, RISK_MM, RainTomorrow], file_type=parquet @@ -651,12 +651,12 @@ mod tests { │ CoalescePartitionsExec │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3] t1:[p0,p1,p2,p3] + ┌───── Stage 2 ── Tasks: t0:[p0..p3] t1:[p0..p3] │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[] │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 1] => NetworkShuffleExec: output_partitions=4, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7] t1:[p0,p1,p2,p3,p4,p5,p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p7] t1:[p0..p7] │ RepartitionExec: partitioning=Hash([RainToday@0, WindGustDir@1], 8), input_partitions=4 │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[] diff --git a/src/stage.rs b/src/stage.rs index 65bf93e..0ed1043 100644 --- a/src/stage.rs +++ b/src/stage.rs @@ -5,7 +5,7 @@ use datafusion::error::Result; use datafusion::execution::TaskContext; use datafusion::physical_plan::display::DisplayableExecutionPlan; use datafusion::physical_plan::{ExecutionPlan, ExecutionPlanProperties, displayable}; -use itertools::{Either, Itertools}; +use itertools::Either; use std::collections::VecDeque; use std::sync::Arc; use url::Url; @@ -313,9 +313,12 @@ fn format_tasks_for_stage(n_tasks: usize, head: &Arc) -> Stri let mut off = 0; for i in 0..n_tasks { result += &format!("t{i}:["); - result += &(off..(off + input_partitions)) - .map(|v| format!("p{v}")) - .join(","); + let end = off + input_partitions - 1; + if input_partitions == 1 { + result += &format!("p{off}"); + } else { + result += &format!("p{off}..p{end}"); + } result += "] "; off += if hash_shuffle { 0 } else { input_partitions } } diff --git a/tests/custom_extension_codec.rs b/tests/custom_extension_codec.rs index 41c3569..47134e6 100644 --- a/tests/custom_extension_codec.rs +++ b/tests/custom_extension_codec.rs @@ -66,12 +66,12 @@ mod tests { │ RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=10 │ [Stage 2] => NetworkShuffleExec: output_partitions=10, input_tasks=10 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] t3:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] t4:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] t5:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] t6:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] t7:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] t8:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] t9:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] + ┌───── Stage 2 ── Tasks: t0:[p0..p9] t1:[p0..p9] t2:[p0..p9] t3:[p0..p9] t4:[p0..p9] t5:[p0..p9] t6:[p0..p9] t7:[p0..p9] t8:[p0..p9] t9:[p0..p9] │ RepartitionExec: partitioning=Hash([], 10), input_partitions=1 │ SortExec: expr=[numbers@0 DESC NULLS LAST], preserve_partitioning=[false] │ [Stage 1] => NetworkShuffleExec: output_partitions=1, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] + ┌───── Stage 1 ── Tasks: t0:[p0..p9] │ RepartitionExec: partitioning=Hash([numbers@0], 10), input_partitions=1 │ FilterExec: numbers@0 > 1 │ Int64ListExec: length=6 diff --git a/tests/distributed_aggregation.rs b/tests/distributed_aggregation.rs index 0027dc8..1743bf5 100644 --- a/tests/distributed_aggregation.rs +++ b/tests/distributed_aggregation.rs @@ -55,7 +55,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2] t1:[p0,p1,p2] + ┌───── Stage 1 ── Tasks: t0:[p0..p2] t1:[p0..p2] │ RepartitionExec: partitioning=Hash([RainToday@0], 3), input_partitions=2 │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] │ PartitionIsolatorExec: t0:[p0,p1,__] t1:[__,__,p0] @@ -132,13 +132,13 @@ mod tests { │ CoalescePartitionsExec │ [Stage 2] => NetworkCoalesceExec: output_partitions=18, input_tasks=6 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2] t1:[p0,p1,p2] t2:[p0,p1,p2] t3:[p0,p1,p2] t4:[p0,p1,p2] t5:[p0,p1,p2] + ┌───── Stage 2 ── Tasks: t0:[p0..p2] t1:[p0..p2] t2:[p0..p2] t3:[p0..p2] t4:[p0..p2] t5:[p0..p2] │ ProjectionExec: expr=[count(Int64(1))@1 as count(*), RainToday@0 as RainToday] │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 1] => NetworkShuffleExec: output_partitions=3, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 1 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] │ RepartitionExec: partitioning=Hash([RainToday@0], 18), input_partitions=1 │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday], aggr=[count(Int64(1))] │ PartitionIsolatorExec: t0:[p0,__,__] t1:[__,p0,__] t2:[__,__,p0] diff --git a/tests/stateful_execution_plan.rs b/tests/stateful_execution_plan.rs index 8339c36..cb4d0e4 100644 --- a/tests/stateful_execution_plan.rs +++ b/tests/stateful_execution_plan.rs @@ -70,12 +70,12 @@ mod tests { │ RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=10 │ [Stage 2] => NetworkShuffleExec: output_partitions=10, input_tasks=10 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] t3:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] t4:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] t5:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] t6:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] t7:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] t8:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] t9:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] + ┌───── Stage 2 ── Tasks: t0:[p0..p9] t1:[p0..p9] t2:[p0..p9] t3:[p0..p9] t4:[p0..p9] t5:[p0..p9] t6:[p0..p9] t7:[p0..p9] t8:[p0..p9] t9:[p0..p9] │ RepartitionExec: partitioning=Hash([], 10), input_partitions=1 │ SortExec: expr=[numbers@0 DESC NULLS LAST], preserve_partitioning=[false] │ [Stage 1] => NetworkShuffleExec: output_partitions=1, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9] + ┌───── Stage 1 ── Tasks: t0:[p0..p9] │ RepartitionExec: partitioning=Hash([numbers@0], 10), input_partitions=1 │ FilterExec: numbers@0 > 1 │ StatefulInt64ListExec: length=6 diff --git a/tests/tpch_validation_test.rs b/tests/tpch_validation_test.rs index 8d8df67..e0df1c1 100644 --- a/tests/tpch_validation_test.rs +++ b/tests/tpch_validation_test.rs @@ -31,14 +31,14 @@ mod tests { │ SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST] │ [Stage 2] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus, sum(lineitem.l_quantity)@2 as sum_qty, sum(lineitem.l_extendedprice)@3 as sum_base_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@4 as sum_disc_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax)@5 as sum_charge, avg(lineitem.l_quantity)@6 as avg_qty, avg(lineitem.l_extendedprice)@7 as avg_price, avg(lineitem.l_discount)@8 as avg_disc, count(Int64(1))@9 as count_order] │ AggregateExec: mode=FinalPartitioned, gby=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(Int64(1))] │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 1 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] │ RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 24), input_partitions=2 │ AggregateExec: mode=Partial, gby=[l_returnflag@5 as l_returnflag, l_linestatus@6 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(Int64(1))] │ ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as __common_expr_1, l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] @@ -54,14 +54,14 @@ mod tests { │ SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST], metrics=[output_rows=4, elapsed_compute=] │ [Stage 2] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=4, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ ProjectionExec: expr=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus, sum(lineitem.l_quantity)@2 as sum_qty, sum(lineitem.l_extendedprice)@3 as sum_base_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@4 as sum_disc_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax)@5 as sum_charge, avg(lineitem.l_quantity)@6 as avg_qty, avg(lineitem.l_extendedprice)@7 as avg_price, avg(lineitem.l_discount)@8 as avg_disc, count(Int64(1))@9 as count_order], metrics=[output_rows=4, elapsed_compute=] │ AggregateExec: mode=FinalPartitioned, gby=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(Int64(1))], metrics=[output_rows=4, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=24, elapsed_compute=] │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 1 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..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=24, 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=591856, elapsed_compute=] @@ -82,7 +82,7 @@ mod tests { │ SortPreservingMergeExec: [s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST] │ [Stage 7] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[s_acctbal@5 as s_acctbal, s_name@2 as s_name, n_name@7 as n_name, p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_address@3 as s_address, s_phone@4 as s_phone, s_comment@6 as s_comment] │ CoalesceBatchesExec: target_batch_size=8192 @@ -92,7 +92,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 3 ── Tasks: t0:[p0..p23] │ RepartitionExec: partitioning=Hash([p_partkey@0, ps_supplycost@7], 24), input_partitions=6 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@9)], projection=[p_partkey@1, p_mfgr@2, s_name@3, s_address@4, s_phone@5, s_acctbal@6, s_comment@7, ps_supplycost@8, n_name@9] @@ -114,26 +114,26 @@ mod tests { │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/partsupp/1.parquet:.., /testdata/tpch/data/partsupp/10.parquet:.., /testdata/tpch/data/partsupp/11.parquet:..], [/testdata/tpch/data/partsupp/11.parquet:.., /testdata/tpch/data/partsupp/12.parquet:.., /testdata/tpch/data/partsupp/13.parquet:.., /testdata/tpch/data/partsupp/14.parquet:..], [/testdata/tpch/data/partsupp/14.parquet:.., /testdata/tpch/data/partsupp/15.parquet:.., /testdata/tpch/data/partsupp/16.parquet:.., /testdata/tpch/data/partsupp/2.parquet:..], [/testdata/tpch/data/partsupp/2.parquet:.., /testdata/tpch/data/partsupp/3.parquet:.., /testdata/tpch/data/partsupp/4.parquet:..], [/testdata/tpch/data/partsupp/4.parquet:.., /testdata/tpch/data/partsupp/5.parquet:.., /testdata/tpch/data/partsupp/6.parquet:.., /testdata/tpch/data/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/region/1.parquet, /testdata/tpch/data/region/10.parquet, /testdata/tpch/data/region/11.parquet], [/testdata/tpch/data/region/12.parquet, /testdata/tpch/data/region/13.parquet, /testdata/tpch/data/region/14.parquet], [/testdata/tpch/data/region/15.parquet, /testdata/tpch/data/region/16.parquet, /testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.parquet, /testdata/tpch/data/region/4.parquet, /testdata/tpch/data/region/5.parquet], [/testdata/tpch/data/region/6.parquet, /testdata/tpch/data/region/7.parquet, /testdata/tpch/data/region/8.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = EUROPE, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= EUROPE AND EUROPE <= r_name_max@1, required_guarantees=[r_name in (EUROPE)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@1] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_mfgr, p_type, p_size], file_type=parquet, predicate=p_size@3 = 15 AND p_type@2 LIKE %BRASS, pruning_predicate=p_size_null_count@2 != row_count@3 AND p_size_min@0 <= 15 AND 15 <= p_size_max@1, required_guarantees=[p_size in (15)] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 6 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] │ RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 24), input_partitions=6 │ ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] │ AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 5 ── Tasks: t0:[p0..p17] │ RepartitionExec: partitioning=Hash([ps_partkey@0], 18), input_partitions=6 │ AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] │ CoalesceBatchesExec: target_batch_size=8192 @@ -152,7 +152,7 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/supplier/1.parquet, /testdata/tpch/data/supplier/10.parquet, /testdata/tpch/data/supplier/11.parquet], [/testdata/tpch/data/supplier/12.parquet, /testdata/tpch/data/supplier/13.parquet, /testdata/tpch/data/supplier/14.parquet], [/testdata/tpch/data/supplier/15.parquet, /testdata/tpch/data/supplier/16.parquet, /testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet, /testdata/tpch/data/supplier/4.parquet, /testdata/tpch/data/supplier/5.parquet], [/testdata/tpch/data/supplier/6.parquet, /testdata/tpch/data/supplier/7.parquet, /testdata/tpch/data/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/partsupp/1.parquet:.., /testdata/tpch/data/partsupp/10.parquet:.., /testdata/tpch/data/partsupp/11.parquet:..], [/testdata/tpch/data/partsupp/11.parquet:.., /testdata/tpch/data/partsupp/12.parquet:.., /testdata/tpch/data/partsupp/13.parquet:.., /testdata/tpch/data/partsupp/14.parquet:..], [/testdata/tpch/data/partsupp/14.parquet:.., /testdata/tpch/data/partsupp/15.parquet:.., /testdata/tpch/data/partsupp/16.parquet:.., /testdata/tpch/data/partsupp/2.parquet:..], [/testdata/tpch/data/partsupp/2.parquet:.., /testdata/tpch/data/partsupp/3.parquet:.., /testdata/tpch/data/partsupp/4.parquet:..], [/testdata/tpch/data/partsupp/4.parquet:.., /testdata/tpch/data/partsupp/5.parquet:.., /testdata/tpch/data/partsupp/6.parquet:.., /testdata/tpch/data/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] @@ -165,7 +165,7 @@ mod tests { │ SortPreservingMergeExec: [s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST], metrics=[output_rows=11264, elapsed_compute=] │ [Stage 7] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..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=11264, 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=11264, elapsed_compute=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=11264, elapsed_compute=] @@ -175,7 +175,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=11985, elapsed_compute=] │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 3 ── Tasks: t0:[p0..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=16128, elapsed_compute=] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(r_regionkey@0, n_regionkey@9)], projection=[p_partkey@1, p_mfgr@2, s_name@3, s_address@4, s_phone@5, s_acctbal@6, s_comment@7, ps_supplycost@8, n_name@9], metrics=[output_rows=16128, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] @@ -197,26 +197,26 @@ mod tests { │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] │ 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=80000, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16, elapsed_compute=] │ FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0], metrics=[output_rows=16, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] │ 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=80, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=73, elapsed_compute=] │ FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@1], metrics=[output_rows=73, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] │ 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=20000, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 6 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..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=11985, elapsed_compute=] │ AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)], metrics=[output_rows=11985, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=11985, elapsed_compute=] │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 5 ── Tasks: t0:[p0..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=11985, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=4177920, elapsed_compute=] @@ -235,7 +235,7 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/supplier/1.parquet, /testdata/tpch/data/supplier/10.parquet, /testdata/tpch/data/supplier/11.parquet], [/testdata/tpch/data/supplier/12.parquet, /testdata/tpch/data/supplier/13.parquet, /testdata/tpch/data/supplier/14.parquet], [/testdata/tpch/data/supplier/15.parquet, /testdata/tpch/data/supplier/16.parquet, /testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet, /testdata/tpch/data/supplier/4.parquet, /testdata/tpch/data/supplier/5.parquet], [/testdata/tpch/data/supplier/6.parquet, /testdata/tpch/data/supplier/7.parquet, /testdata/tpch/data/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, metrics=[output_rows=1000, 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=80000, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 4 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16, elapsed_compute=] │ FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0], metrics=[output_rows=16, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] @@ -253,14 +253,14 @@ mod tests { │ SortPreservingMergeExec: [revenue@1 DESC, o_orderdate@2 ASC NULLS LAST] │ [Stage 3] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[revenue@1 DESC, o_orderdate@2 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[l_orderkey@0 as l_orderkey, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@3 as revenue, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority] │ AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 2 ── Tasks: t0:[p0..p23] │ RepartitionExec: partitioning=Hash([l_orderkey@0, o_orderdate@1, o_shippriority@2], 24), input_partitions=6 │ AggregateExec: mode=Partial, gby=[l_orderkey@2 as l_orderkey, o_orderdate@0 as o_orderdate, o_shippriority@1 as o_shippriority], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ CoalesceBatchesExec: target_batch_size=8192 @@ -277,7 +277,7 @@ mod tests { │ FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 > 1995-03-15 AND DynamicFilterPhysicalExpr [ true ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 > 1995-03-15, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] @@ -289,14 +289,14 @@ mod tests { │ SortPreservingMergeExec: [revenue@1 DESC, o_orderdate@2 ASC NULLS LAST], metrics=[output_rows=1216, elapsed_compute=] │ [Stage 3] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[revenue@1 DESC, o_orderdate@2 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=1216, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ ProjectionExec: expr=[l_orderkey@0 as l_orderkey, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@3 as revenue, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority], metrics=[output_rows=1216, elapsed_compute=] │ AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=1216, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1216, elapsed_compute=] │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 2 ── Tasks: t0:[p0..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=1216, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=3321, elapsed_compute=] @@ -313,7 +313,7 @@ mod tests { │ FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2], metrics=[output_rows=324322, elapsed_compute=] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=600572, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=3111, elapsed_compute=] │ FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0], metrics=[output_rows=3111, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] @@ -331,14 +331,14 @@ mod tests { │ SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST] │ [Stage 3] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[o_orderpriority@0 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[o_orderpriority@0 as o_orderpriority, count(Int64(1))@1 as order_count] │ AggregateExec: mode=FinalPartitioned, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))] │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 2 ── Tasks: t0:[p0..p23] │ RepartitionExec: partitioning=Hash([o_orderpriority@0], 24), input_partitions=6 │ AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))] │ CoalesceBatchesExec: target_batch_size=8192 @@ -349,7 +349,7 @@ mod tests { │ FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderdate, o_orderpriority], file_type=parquet, predicate=o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1993-07-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 < 1993-10-01, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] @@ -362,14 +362,14 @@ mod tests { │ SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST], metrics=[output_rows=5, elapsed_compute=] │ [Stage 3] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[o_orderpriority@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=5, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ ProjectionExec: expr=[o_orderpriority@0 as o_orderpriority, count(Int64(1))@1 as order_count], metrics=[output_rows=5, elapsed_compute=] │ AggregateExec: mode=FinalPartitioned, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))], metrics=[output_rows=5, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=30, elapsed_compute=] │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 2 ── Tasks: t0:[p0..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=30, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5093, elapsed_compute=] @@ -380,7 +380,7 @@ mod tests { │ FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2], metrics=[output_rows=5552, elapsed_compute=] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=150000, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=379809, elapsed_compute=] │ FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0], metrics=[output_rows=379809, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] @@ -398,14 +398,14 @@ mod tests { │ SortPreservingMergeExec: [revenue@1 DESC] │ [Stage 4] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[revenue@1 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[n_name@0 as n_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as revenue] │ AggregateExec: mode=FinalPartitioned, gby=[n_name@0 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 3 ── Tasks: t0:[p0..p23] │ RepartitionExec: partitioning=Hash([n_name@0], 24), input_partitions=6 │ AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ CoalesceBatchesExec: target_batch_size=8192 @@ -433,13 +433,13 @@ 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 ] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/region/1.parquet, /testdata/tpch/data/region/10.parquet, /testdata/tpch/data/region/11.parquet], [/testdata/tpch/data/region/12.parquet, /testdata/tpch/data/region/13.parquet, /testdata/tpch/data/region/14.parquet], [/testdata/tpch/data/region/15.parquet, /testdata/tpch/data/region/16.parquet, /testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.parquet, /testdata/tpch/data/region/4.parquet, /testdata/tpch/data/region/5.parquet], [/testdata/tpch/data/region/6.parquet, /testdata/tpch/data/region/7.parquet, /testdata/tpch/data/region/8.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = ASIA, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= ASIA AND ASIA <= r_name_max@1, required_guarantees=[r_name in (ASIA)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] @@ -452,14 +452,14 @@ mod tests { │ SortPreservingMergeExec: [revenue@1 DESC], metrics=[output_rows=5, elapsed_compute=] │ [Stage 4] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[revenue@1 DESC], preserve_partitioning=[true], metrics=[output_rows=5, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ ProjectionExec: expr=[n_name@0 as n_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as revenue], metrics=[output_rows=5, elapsed_compute=] │ AggregateExec: mode=FinalPartitioned, gby=[n_name@0 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=5, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=30, elapsed_compute=] │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 3 ── Tasks: t0:[p0..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=30, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=221440, elapsed_compute=] @@ -487,13 +487,13 @@ 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 ], metrics=[output_rows=15000, 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=600572, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16, elapsed_compute=] │ FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0], metrics=[output_rows=16, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] │ 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=80, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=22958, 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=22958, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] @@ -513,7 +513,7 @@ mod tests { │ CoalescePartitionsExec │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2, projection=[l_extendedprice@1, l_discount@2] @@ -529,7 +529,7 @@ mod tests { │ CoalescePartitionsExec, metrics=[output_rows=8, elapsed_compute=] │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)], metrics=[output_rows=8, elapsed_compute=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=11618, 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=11618, elapsed_compute=] @@ -548,14 +548,14 @@ mod tests { │ SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST] │ [Stage 8] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year, sum(shipping.volume)@3 as revenue] │ AggregateExec: mode=FinalPartitioned, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)] │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 7] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 7 ── Tasks: t0:[p0..p23] │ RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 24), input_partitions=6 │ AggregateExec: mode=Partial, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)] │ ProjectionExec: expr=[n_name@4 as supp_nation, n_name@0 as cust_nation, date_part(YEAR, l_shipdate@3) as l_year, l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as volume] @@ -576,24 +576,24 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = GERMANY OR n_name@1 = FRANCE, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1 OR n_name_null_count@2 != row_count@3 AND n_name_min@0 <= FRANCE AND FRANCE <= n_name_max@1, required_guarantees=[n_name in (FRANCE, GERMANY)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = FRANCE OR n_name@1 = GERMANY, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= FRANCE AND FRANCE <= n_name_max@1 OR n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1, required_guarantees=[n_name in (FRANCE, GERMANY)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([c_custkey@0], 6), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([o_custkey@4], 6), input_partitions=6 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@1, o_orderkey@0)], projection=[s_nationkey@0, l_extendedprice@2, l_discount@3, l_shipdate@4, o_custkey@6] @@ -602,7 +602,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 4 ── Tasks: t0:[p0..p17] │ RepartitionExec: partitioning=Hash([l_orderkey@1], 18), input_partitions=6 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5, l_shipdate@6] @@ -612,7 +612,7 @@ mod tests { │ FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 AND DynamicFilterPhysicalExpr [ true ], pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1995-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 <= 1996-12-31, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 5 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] │ RepartitionExec: partitioning=Hash([o_orderkey@0], 18), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ] @@ -624,14 +624,14 @@ mod tests { │ SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST], metrics=[output_rows=4, elapsed_compute=] │ [Stage 8] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=4, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ ProjectionExec: expr=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year, sum(shipping.volume)@3 as revenue], metrics=[output_rows=4, elapsed_compute=] │ AggregateExec: mode=FinalPartitioned, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)], metrics=[output_rows=4, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=24, elapsed_compute=] │ [Stage 7] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 7 ── Tasks: t0:[p0..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=24, 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=164608, elapsed_compute=] @@ -652,24 +652,24 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=182762, elapsed_compute=] │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=32, elapsed_compute=] │ FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE, metrics=[output_rows=32, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] │ 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=400, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=32, elapsed_compute=] │ FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY, metrics=[output_rows=32, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] │ 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=400, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([c_custkey@0], 6), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] │ 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=15000, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..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=182762, elapsed_compute=] │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@1, o_orderkey@0)], projection=[s_nationkey@0, l_extendedprice@2, l_discount@3, l_shipdate@4, o_custkey@6], metrics=[output_rows=182762, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] @@ -678,7 +678,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=150000, elapsed_compute=] │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 4 ── Tasks: t0:[p0..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=182762, elapsed_compute=] │ HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5, l_shipdate@6], metrics=[output_rows=182762, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] @@ -688,7 +688,7 @@ mod tests { │ FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31, metrics=[output_rows=182762, elapsed_compute=] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=600572, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 5 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] │ RepartitionExec: partitioning=Hash([o_orderkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] │ 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=150000, 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=] @@ -705,14 +705,14 @@ mod tests { │ SortPreservingMergeExec: [o_year@0 ASC NULLS LAST] │ [Stage 8] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[o_year@0 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[o_year@0 as o_year, sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END)@1 / sum(all_nations.volume)@2 as mkt_share] │ AggregateExec: mode=FinalPartitioned, gby=[o_year@0 as o_year], aggr=[sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)] │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 7] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 7 ── Tasks: t0:[p0..p23] │ RepartitionExec: partitioning=Hash([o_year@0], 24), input_partitions=6 │ AggregateExec: mode=Partial, gby=[o_year@0 as o_year], aggr=[sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)] │ ProjectionExec: expr=[date_part(YEAR, o_orderdate@2) as o_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume, n_name@3 as nation] @@ -737,13 +737,13 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/region/1.parquet, /testdata/tpch/data/region/10.parquet, /testdata/tpch/data/region/11.parquet], [/testdata/tpch/data/region/12.parquet, /testdata/tpch/data/region/13.parquet, /testdata/tpch/data/region/14.parquet], [/testdata/tpch/data/region/15.parquet, /testdata/tpch/data/region/16.parquet, /testdata/tpch/data/region/2.parquet], [/testdata/tpch/data/region/3.parquet, /testdata/tpch/data/region/4.parquet, /testdata/tpch/data/region/5.parquet], [/testdata/tpch/data/region/6.parquet, /testdata/tpch/data/region/7.parquet, /testdata/tpch/data/region/8.parquet], ...]}, projection=[r_regionkey, r_name], file_type=parquet, predicate=r_name@1 = AMERICA, pruning_predicate=r_name_null_count@2 != row_count@3 AND r_name_min@0 <= AMERICA AND AMERICA <= r_name_max@1, required_guarantees=[r_name in (AMERICA)] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([o_custkey@3], 6), input_partitions=6 │ ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@4 as s_nationkey, o_custkey@0 as o_custkey, o_orderdate@1 as o_orderdate] │ CoalesceBatchesExec: target_batch_size=8192 @@ -753,14 +753,14 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 2 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] │ RepartitionExec: partitioning=Hash([o_orderkey@0], 18), input_partitions=2 │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=parquet, predicate=o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31, pruning_predicate=o_orderdate_null_count@1 != row_count@2 AND o_orderdate_max@0 >= 1995-01-01 AND o_orderdate_null_count@1 != row_count@2 AND o_orderdate_min@3 <= 1996-12-31, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 4 ── Tasks: t0:[p0..p17] │ RepartitionExec: partitioning=Hash([l_orderkey@0], 18), input_partitions=6 │ ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@0 as s_nationkey] │ CoalesceBatchesExec: target_batch_size=8192 @@ -773,13 +773,13 @@ mod tests { │ [Stage 3] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 3 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: p_type@1 = ECONOMY ANODIZED STEEL, projection=[p_partkey@0] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_type], file_type=parquet, predicate=p_type@1 = ECONOMY ANODIZED STEEL, pruning_predicate=p_type_null_count@2 != row_count@3 AND p_type_min@0 <= ECONOMY ANODIZED STEEL AND ECONOMY ANODIZED STEEL <= p_type_max@1, required_guarantees=[p_type in (ECONOMY ANODIZED STEEL)] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([c_custkey@0], 6), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_nationkey], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ] @@ -791,14 +791,14 @@ mod tests { │ SortPreservingMergeExec: [o_year@0 ASC NULLS LAST], metrics=[output_rows=2, elapsed_compute=] │ [Stage 8] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[o_year@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=2, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ ProjectionExec: expr=[o_year@0 as o_year, sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END)@1 / sum(all_nations.volume)@2 as mkt_share], metrics=[output_rows=2, elapsed_compute=] │ AggregateExec: mode=FinalPartitioned, gby=[o_year@0 as o_year], aggr=[sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)], metrics=[output_rows=2, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=12, elapsed_compute=] │ [Stage 7] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 7 ── Tasks: t0:[p0..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=12, 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=1155072, elapsed_compute=] @@ -823,13 +823,13 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=15000, elapsed_compute=] │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16, elapsed_compute=] │ FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0], metrics=[output_rows=16, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] │ 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=80, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..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=1429, elapsed_compute=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1429, elapsed_compute=] @@ -839,14 +839,14 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=4485, elapsed_compute=] │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 2 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..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=45624, elapsed_compute=] │ FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31, metrics=[output_rows=45624, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] │ 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=150000, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 4 ── Tasks: t0:[p0..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=4485, elapsed_compute=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=4485, elapsed_compute=] @@ -859,13 +859,13 @@ mod tests { │ [Stage 3] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] │ 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=600572, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 3 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=147, elapsed_compute=] │ FilterExec: p_type@1 = ECONOMY ANODIZED STEEL, projection=[p_partkey@0], metrics=[output_rows=147, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] │ 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=20000, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([c_custkey@0], 6), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] │ 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=15000, 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=] @@ -882,14 +882,14 @@ mod tests { │ SortPreservingMergeExec: [nation@0 ASC NULLS LAST, o_year@1 DESC] │ [Stage 7] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[nation@0 ASC NULLS LAST, o_year@1 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[nation@0 as nation, o_year@1 as o_year, sum(profit.amount)@2 as sum_profit] │ AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 6 ── Tasks: t0:[p0..p23] │ RepartitionExec: partitioning=Hash([nation@0, o_year@1], 24), input_partitions=6 │ AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] │ ProjectionExec: expr=[n_name@0 as nation, date_part(YEAR, o_orderdate@5) as o_year, l_extendedprice@2 * (Some(1),20,0 - l_discount@3) - ps_supplycost@4 * l_quantity@1 as amount] @@ -905,12 +905,12 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([o_orderkey@0], 6), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderdate], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([l_orderkey@0], 6), input_partitions=6 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, ps_suppkey@1), (l_partkey@1, ps_partkey@0)], projection=[l_orderkey@0, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@9] @@ -919,7 +919,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 3 ── Tasks: t0:[p0..p17] │ RepartitionExec: partitioning=Hash([l_suppkey@2, l_partkey@1], 18), input_partitions=6 │ ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_partkey@2 as l_partkey, l_suppkey@3 as l_suppkey, l_quantity@4 as l_quantity, l_extendedprice@5 as l_extendedprice, l_discount@6 as l_discount, s_nationkey@0 as s_nationkey] │ CoalesceBatchesExec: target_batch_size=8192 @@ -932,13 +932,13 @@ mod tests { │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: p_name@1 LIKE %green%, projection=[p_partkey@0] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_name], file_type=parquet, predicate=p_name@1 LIKE %green% └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 4 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] │ RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 18), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/partsupp/1.parquet:.., /testdata/tpch/data/partsupp/10.parquet:.., /testdata/tpch/data/partsupp/11.parquet:..], [/testdata/tpch/data/partsupp/11.parquet:.., /testdata/tpch/data/partsupp/12.parquet:.., /testdata/tpch/data/partsupp/13.parquet:.., /testdata/tpch/data/partsupp/14.parquet:..], [/testdata/tpch/data/partsupp/14.parquet:.., /testdata/tpch/data/partsupp/15.parquet:.., /testdata/tpch/data/partsupp/16.parquet:.., /testdata/tpch/data/partsupp/2.parquet:..], [/testdata/tpch/data/partsupp/2.parquet:.., /testdata/tpch/data/partsupp/3.parquet:.., /testdata/tpch/data/partsupp/4.parquet:..], [/testdata/tpch/data/partsupp/4.parquet:.., /testdata/tpch/data/partsupp/5.parquet:.., /testdata/tpch/data/partsupp/6.parquet:.., /testdata/tpch/data/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] @@ -950,14 +950,14 @@ mod tests { │ SortPreservingMergeExec: [nation@0 ASC NULLS LAST, o_year@1 DESC], metrics=[output_rows=175, elapsed_compute=] │ [Stage 7] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 7 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[nation@0 ASC NULLS LAST, o_year@1 DESC], preserve_partitioning=[true], metrics=[output_rows=175, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ ProjectionExec: expr=[nation@0 as nation, o_year@1 as o_year, sum(profit.amount)@2 as sum_profit], metrics=[output_rows=175, elapsed_compute=] │ AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)], metrics=[output_rows=175, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1050, elapsed_compute=] │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 6 ── Tasks: t0:[p0..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=1050, 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=514560, elapsed_compute=] @@ -973,12 +973,12 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=32160, elapsed_compute=] │ [Stage 5] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 1 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([o_orderkey@0], 6), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] │ 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=150000, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..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=32160, elapsed_compute=] │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, ps_suppkey@1), (l_partkey@1, ps_partkey@0)], projection=[l_orderkey@0, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@9], metrics=[output_rows=32160, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] @@ -987,7 +987,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=80000, elapsed_compute=] │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 3 ── Tasks: t0:[p0..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=32160, elapsed_compute=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=32160, elapsed_compute=] @@ -1000,13 +1000,13 @@ mod tests { │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] │ 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=600572, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1075, elapsed_compute=] │ FilterExec: p_name@1 LIKE %green%, projection=[p_partkey@0], metrics=[output_rows=1075, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] │ 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=20000, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 4 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] │ RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] │ 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=80000, 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=] @@ -1023,14 +1023,14 @@ mod tests { │ SortPreservingMergeExec: [revenue@2 DESC] │ [Stage 3] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[revenue@2 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@7 as revenue, c_acctbal@2 as c_acctbal, n_name@4 as n_name, c_address@5 as c_address, c_phone@3 as c_phone, c_comment@6 as c_comment] │ AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@2 as c_acctbal, c_phone@3 as c_phone, n_name@4 as n_name, c_address@5 as c_address, c_comment@6 as c_comment], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 2 ── Tasks: t0:[p0..p23] │ RepartitionExec: partitioning=Hash([c_custkey@0, c_name@1, c_acctbal@2, c_phone@3, n_name@4, c_address@5, c_comment@6], 24), input_partitions=6 │ AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@4 as c_acctbal, c_phone@3 as c_phone, n_name@8 as n_name, c_address@2 as c_address, c_comment@5 as c_comment], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ ProjectionExec: expr=[c_custkey@1 as c_custkey, c_name@2 as c_name, c_address@3 as c_address, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, l_extendedprice@7 as l_extendedprice, l_discount@8 as l_discount, n_name@0 as n_name] @@ -1051,7 +1051,7 @@ mod tests { │ FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], file_type=parquet, predicate=l_returnflag@3 = R AND DynamicFilterPhysicalExpr [ true ], pruning_predicate=l_returnflag_null_count@2 != row_count@3 AND l_returnflag_min@0 <= R AND R <= l_returnflag_max@1, required_guarantees=[l_returnflag in (R)] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] @@ -1064,14 +1064,14 @@ mod tests { │ SortPreservingMergeExec: [revenue@2 DESC], metrics=[output_rows=3767, elapsed_compute=] │ [Stage 3] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[revenue@2 DESC], preserve_partitioning=[true], metrics=[output_rows=3767, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@7 as revenue, c_acctbal@2 as c_acctbal, n_name@4 as n_name, c_address@5 as c_address, c_phone@3 as c_phone, c_comment@6 as c_comment], metrics=[output_rows=3767, elapsed_compute=] │ AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@2 as c_acctbal, c_phone@3 as c_phone, n_name@4 as n_name, c_address@5 as c_address, c_comment@6 as c_comment], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=3767, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=4648, elapsed_compute=] │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 2 ── Tasks: t0:[p0..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=4648, 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=183024, elapsed_compute=] @@ -1092,7 +1092,7 @@ mod tests { │ FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2], metrics=[output_rows=148301, elapsed_compute=] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=600572, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5677, 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=5677, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] @@ -1140,13 +1140,13 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/supplier/1.parquet, /testdata/tpch/data/supplier/10.parquet, /testdata/tpch/data/supplier/11.parquet], [/testdata/tpch/data/supplier/12.parquet, /testdata/tpch/data/supplier/13.parquet, /testdata/tpch/data/supplier/14.parquet], [/testdata/tpch/data/supplier/15.parquet, /testdata/tpch/data/supplier/16.parquet, /testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet, /testdata/tpch/data/supplier/4.parquet, /testdata/tpch/data/supplier/5.parquet], [/testdata/tpch/data/supplier/6.parquet, /testdata/tpch/data/supplier/7.parquet, /testdata/tpch/data/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/partsupp/1.parquet:.., /testdata/tpch/data/partsupp/10.parquet:.., /testdata/tpch/data/partsupp/11.parquet:..], [/testdata/tpch/data/partsupp/11.parquet:.., /testdata/tpch/data/partsupp/12.parquet:.., /testdata/tpch/data/partsupp/13.parquet:.., /testdata/tpch/data/partsupp/14.parquet:..], [/testdata/tpch/data/partsupp/14.parquet:.., /testdata/tpch/data/partsupp/15.parquet:.., /testdata/tpch/data/partsupp/16.parquet:.., /testdata/tpch/data/partsupp/2.parquet:..], [/testdata/tpch/data/partsupp/2.parquet:.., /testdata/tpch/data/partsupp/3.parquet:.., /testdata/tpch/data/partsupp/4.parquet:..], [/testdata/tpch/data/partsupp/4.parquet:.., /testdata/tpch/data/partsupp/5.parquet:.., /testdata/tpch/data/partsupp/6.parquet:.., /testdata/tpch/data/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ ps_suppkey@1 >= 1 AND ps_suppkey@1 <= 1000 OR ps_suppkey@1 >= 1 AND ps_suppkey@1 <= 1000 OR ps_suppkey@1 >= 1 AND ps_suppkey@1 <= 1000 OR ps_suppkey@1 >= 1 AND ps_suppkey@1 <= 1000 OR ps_suppkey@1 >= 1 AND ps_suppkey@1 <= 1000 OR ps_suppkey@1 >= 1 AND ps_suppkey@1 <= 1000 ], pruning_predicate=ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_max@0 >= 1 AND ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_min@3 <= 1000 OR ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_max@0 >= 1 AND ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_min@3 <= 1000 OR ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_max@0 >= 1 AND ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_min@3 <= 1000 OR ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_max@0 >= 1 AND ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_min@3 <= 1000 OR ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_max@0 >= 1 AND ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_min@3 <= 1000 OR ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_max@0 >= 1 AND ps_suppkey_null_count@1 != row_count@2 AND ps_suppkey_min@3 <= 1000, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = GERMANY, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= GERMANY AND GERMANY <= n_name_max@1, required_guarantees=[n_name in (GERMANY)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] @@ -1189,13 +1189,13 @@ mod tests { │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/supplier/1.parquet, /testdata/tpch/data/supplier/10.parquet, /testdata/tpch/data/supplier/11.parquet], [/testdata/tpch/data/supplier/12.parquet, /testdata/tpch/data/supplier/13.parquet, /testdata/tpch/data/supplier/14.parquet], [/testdata/tpch/data/supplier/15.parquet, /testdata/tpch/data/supplier/16.parquet, /testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet, /testdata/tpch/data/supplier/4.parquet, /testdata/tpch/data/supplier/5.parquet], [/testdata/tpch/data/supplier/6.parquet, /testdata/tpch/data/supplier/7.parquet, /testdata/tpch/data/supplier/8.parquet], ...]}, projection=[s_suppkey, s_nationkey], file_type=parquet, metrics=[output_rows=1000, 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=80000, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16, elapsed_compute=] │ FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0], metrics=[output_rows=16, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] │ 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=400, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16, elapsed_compute=] │ FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0], metrics=[output_rows=16, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] @@ -1213,14 +1213,14 @@ mod tests { │ SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST] │ [Stage 4] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[l_shipmode@0 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[l_shipmode@0 as l_shipmode, sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@1 as high_line_count, sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@2 as low_line_count] │ AggregateExec: mode=FinalPartitioned, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 3 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] │ RepartitionExec: partitioning=Hash([l_shipmode@0], 24), input_partitions=6 │ AggregateExec: mode=Partial, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] │ CoalesceBatchesExec: target_batch_size=8192 @@ -1230,14 +1230,14 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 1 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] │ RepartitionExec: partitioning=Hash([l_orderkey@0], 18), input_partitions=2 │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, projection=[l_orderkey@0, l_shipmode@4] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], file_type=parquet, predicate=(l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, pruning_predicate=(l_shipmode_null_count@2 != row_count@3 AND l_shipmode_min@0 <= MAIL AND MAIL <= l_shipmode_max@1 OR l_shipmode_null_count@2 != row_count@3 AND l_shipmode_min@0 <= SHIP AND SHIP <= l_shipmode_max@1) AND l_receiptdate_null_count@5 != row_count@3 AND l_receiptdate_max@4 >= 1994-01-01 AND l_receiptdate_null_count@5 != row_count@3 AND l_receiptdate_min@6 < 1995-01-01, required_guarantees=[l_shipmode in (MAIL, SHIP)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 2 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] │ RepartitionExec: partitioning=Hash([o_orderkey@0], 18), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_orderpriority], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] @@ -1249,14 +1249,14 @@ mod tests { │ SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST], metrics=[output_rows=2, elapsed_compute=] │ [Stage 4] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[l_shipmode@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=2, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ ProjectionExec: expr=[l_shipmode@0 as l_shipmode, sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@1 as high_line_count, sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@2 as low_line_count], metrics=[output_rows=2, elapsed_compute=] │ AggregateExec: mode=FinalPartitioned, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)], metrics=[output_rows=2, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=36, elapsed_compute=] │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 3 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..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=36, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=3155, elapsed_compute=] @@ -1266,14 +1266,14 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=150000, elapsed_compute=] │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 1 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..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=3155, elapsed_compute=] │ FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, projection=[l_orderkey@0, l_shipmode@4], metrics=[output_rows=3155, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] │ 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=600572, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 2 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] │ RepartitionExec: partitioning=Hash([o_orderkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] │ 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=150000, 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=] @@ -1290,14 +1290,14 @@ mod tests { │ SortPreservingMergeExec: [custdist@1 DESC, c_count@0 DESC] │ [Stage 4] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[custdist@1 DESC, c_count@0 DESC], preserve_partitioning=[true] │ ProjectionExec: expr=[c_count@0 as c_count, count(Int64(1))@1 as custdist] │ AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(Int64(1))] │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 3 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] │ RepartitionExec: partitioning=Hash([c_count@0], 24), input_partitions=6 │ AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(Int64(1))] │ ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count] @@ -1305,7 +1305,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 2 ── Tasks: t0:[p0..p17] │ RepartitionExec: partitioning=Hash([c_custkey@0], 18), input_partitions=6 │ AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] │ ProjectionExec: expr=[c_custkey@1 as c_custkey, o_orderkey@0 as o_orderkey] @@ -1315,7 +1315,7 @@ mod tests { │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] @@ -1328,14 +1328,14 @@ mod tests { │ SortPreservingMergeExec: [custdist@1 DESC, c_count@0 DESC], metrics=[output_rows=37, elapsed_compute=] │ [Stage 4] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[custdist@1 DESC, c_count@0 DESC], preserve_partitioning=[true], metrics=[output_rows=37, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ ProjectionExec: expr=[c_count@0 as c_count, count(Int64(1))@1 as custdist], metrics=[output_rows=37, elapsed_compute=] │ AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(Int64(1))], metrics=[output_rows=37, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=568, elapsed_compute=] │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 3 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..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=568, 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=15000, elapsed_compute=] @@ -1343,7 +1343,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=15000, elapsed_compute=] │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 2 ── Tasks: t0:[p0..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=15000, 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=153318, elapsed_compute=] @@ -1353,7 +1353,7 @@ mod tests { │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] │ 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=15000, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=148318, elapsed_compute=] │ FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1], metrics=[output_rows=148318, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] @@ -1373,7 +1373,7 @@ mod tests { │ CoalescePartitionsExec │ [Stage 3] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as __common_expr_1, p_type@0 as p_type] │ CoalesceBatchesExec: target_batch_size=8192 @@ -1383,12 +1383,12 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 1 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] │ RepartitionExec: partitioning=Hash([p_partkey@0], 24), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_type], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 2 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] │ RepartitionExec: partitioning=Hash([l_partkey@0], 24), input_partitions=2 │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01, projection=[l_partkey@0, l_extendedprice@1, l_discount@2] @@ -1404,7 +1404,7 @@ mod tests { │ CoalescePartitionsExec, metrics=[output_rows=24, elapsed_compute=] │ [Stage 3] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..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=24, 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=7630, elapsed_compute=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=7630, elapsed_compute=] @@ -1414,12 +1414,12 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=7630, elapsed_compute=] │ [Stage 2] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 1 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] │ RepartitionExec: partitioning=Hash([p_partkey@0], 24), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] │ 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=20000, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 2 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..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=7630, elapsed_compute=] │ FilterExec: l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01, projection=[l_partkey@0, l_extendedprice@1, l_discount@2], metrics=[output_rows=7630, elapsed_compute=] @@ -1451,14 +1451,14 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[max(revenue0.total_revenue)] │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] │ AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 1 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] │ RepartitionExec: partitioning=Hash([l_suppkey@0], 24), input_partitions=2 │ AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ CoalesceBatchesExec: target_batch_size=8192 @@ -1466,7 +1466,7 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=parquet, predicate=l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, pruning_predicate=l_shipdate_null_count@1 != row_count@2 AND l_shipdate_max@0 >= 1996-01-01 AND l_shipdate_null_count@1 != row_count@2 AND l_shipdate_min@3 < 1996-04-01, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([l_suppkey@0], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] │ CoalesceBatchesExec: target_batch_size=8192 @@ -1494,14 +1494,14 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5825, elapsed_compute=] │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[max(revenue0.total_revenue)], metrics=[output_rows=24, elapsed_compute=] │ ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue], metrics=[output_rows=1000, elapsed_compute=] │ AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], metrics=[output_rows=1000, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5825, elapsed_compute=] │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 1 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..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=5825, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=22830, elapsed_compute=] @@ -1509,7 +1509,7 @@ mod tests { │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] │ 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=600572, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..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=5825, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=22830, elapsed_compute=] @@ -1529,21 +1529,21 @@ mod tests { │ SortPreservingMergeExec: [supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST] │ [Stage 5] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, count(alias1)@3 as supplier_cnt] │ AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[count(alias1)] │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 4 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] │ RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2], 24), input_partitions=6 │ AggregateExec: mode=Partial, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[count(alias1)] │ AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, alias1@3 as alias1], aggr=[] │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 3 ── Tasks: t0:[p0..p17] │ RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 18), input_partitions=6 │ AggregateExec: mode=Partial, gby=[p_brand@1 as p_brand, p_type@2 as p_type, p_size@3 as p_size, ps_suppkey@0 as alias1], aggr=[] │ CoalesceBatchesExec: target_batch_size=8192 @@ -1557,13 +1557,13 @@ mod tests { │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/partsupp/1.parquet:.., /testdata/tpch/data/partsupp/10.parquet:.., /testdata/tpch/data/partsupp/11.parquet:..], [/testdata/tpch/data/partsupp/11.parquet:.., /testdata/tpch/data/partsupp/12.parquet:.., /testdata/tpch/data/partsupp/13.parquet:.., /testdata/tpch/data/partsupp/14.parquet:..], [/testdata/tpch/data/partsupp/14.parquet:.., /testdata/tpch/data/partsupp/15.parquet:.., /testdata/tpch/data/partsupp/16.parquet:.., /testdata/tpch/data/partsupp/2.parquet:..], [/testdata/tpch/data/partsupp/2.parquet:.., /testdata/tpch/data/partsupp/3.parquet:.., /testdata/tpch/data/partsupp/4.parquet:..], [/testdata/tpch/data/partsupp/4.parquet:.., /testdata/tpch/data/partsupp/5.parquet:.., /testdata/tpch/data/partsupp/6.parquet:.., /testdata/tpch/data/partsupp/7.parquet:..], ...]}, projection=[ps_partkey, ps_suppkey], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/supplier/1.parquet, /testdata/tpch/data/supplier/10.parquet, /testdata/tpch/data/supplier/11.parquet], [/testdata/tpch/data/supplier/12.parquet, /testdata/tpch/data/supplier/13.parquet, /testdata/tpch/data/supplier/14.parquet], [/testdata/tpch/data/supplier/15.parquet, /testdata/tpch/data/supplier/16.parquet, /testdata/tpch/data/supplier/2.parquet], [/testdata/tpch/data/supplier/3.parquet, /testdata/tpch/data/supplier/4.parquet, /testdata/tpch/data/supplier/5.parquet], [/testdata/tpch/data/supplier/6.parquet, /testdata/tpch/data/supplier/7.parquet, /testdata/tpch/data/supplier/8.parquet], ...]}, projection=[s_suppkey, s_comment], file_type=parquet, predicate=s_comment@1 LIKE %Customer%Complaints% └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(49), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(14), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(23), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(45), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(19), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(3), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(36), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Int32(9), field: Field { name: "lit", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] @@ -1576,21 +1576,21 @@ mod tests { │ SortPreservingMergeExec: [supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST], metrics=[output_rows=2762, elapsed_compute=] │ [Stage 5] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=2762, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ ProjectionExec: expr=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, count(alias1)@3 as supplier_cnt], metrics=[output_rows=2762, elapsed_compute=] │ AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[count(alias1)], metrics=[output_rows=2762, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=10538, elapsed_compute=] │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 4 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..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=10538, 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=11632, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=11634, elapsed_compute=] │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 3 ── Tasks: t0:[p0..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=11634, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=11635, elapsed_compute=] @@ -1604,13 +1604,13 @@ mod tests { │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] │ 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=80000, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=1, elapsed_compute=] │ FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0], metrics=[output_rows=1, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] │ 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=1000, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=2911, 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=2911, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] @@ -1630,7 +1630,7 @@ mod tests { │ CoalescePartitionsExec │ [Stage 4] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice)] │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * avg(lineitem.l_quantity)@1, projection=[l_extendedprice@1] @@ -1641,7 +1641,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 2 ── Tasks: t0:[p0..p23] │ RepartitionExec: partitioning=Hash([p_partkey@2], 24), input_partitions=6 │ ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, p_partkey@0 as p_partkey] │ CoalesceBatchesExec: target_batch_size=8192 @@ -1650,13 +1650,13 @@ mod tests { │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_quantity, l_extendedprice], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_brand, p_container], file_type=parquet, predicate=p_brand@1 = Brand#23 AND p_container@2 = MED BOX, pruning_predicate=p_brand_null_count@2 != row_count@3 AND p_brand_min@0 <= Brand#23 AND Brand#23 <= p_brand_max@1 AND p_container_null_count@6 != row_count@3 AND p_container_min@4 <= MED BOX AND MED BOX <= p_container_max@5, required_guarantees=[p_brand in (Brand#23), p_container in (MED BOX)] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 3 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] │ RepartitionExec: partitioning=Hash([l_partkey@0], 24), input_partitions=2 │ AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] @@ -1671,7 +1671,7 @@ mod tests { │ CoalescePartitionsExec, metrics=[output_rows=24, elapsed_compute=] │ [Stage 4] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice)], metrics=[output_rows=24, elapsed_compute=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=43, elapsed_compute=] │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * avg(lineitem.l_quantity)@1, projection=[l_extendedprice@1], metrics=[output_rows=43, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] @@ -1682,7 +1682,7 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=118823, elapsed_compute=] │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 2 ── Tasks: t0:[p0..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=555, elapsed_compute=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=555, elapsed_compute=] @@ -1691,13 +1691,13 @@ mod tests { │ [Stage 1] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] │ 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=600572, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=18, elapsed_compute=] │ FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0], metrics=[output_rows=18, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] │ 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=20000, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 3 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..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=118823, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] @@ -1715,13 +1715,13 @@ mod tests { │ SortPreservingMergeExec: [o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST] │ [Stage 8] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST], preserve_partitioning=[true] │ AggregateExec: mode=FinalPartitioned, gby=[c_name@0 as c_name, c_custkey@1 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@3 as o_orderdate, o_totalprice@4 as o_totalprice], aggr=[sum(lineitem.l_quantity)] │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 7] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 7 ── Tasks: t0:[p0..p23] │ RepartitionExec: partitioning=Hash([c_name@0, c_custkey@1, o_orderkey@2, o_orderdate@3, o_totalprice@4], 24), input_partitions=6 │ AggregateExec: mode=Partial, gby=[c_name@1 as c_name, c_custkey@0 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@4 as o_orderdate, o_totalprice@3 as o_totalprice], aggr=[sum(lineitem.l_quantity)] │ CoalesceBatchesExec: target_batch_size=8192 @@ -1735,20 +1735,20 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0] │ AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 1 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..p23] │ RepartitionExec: partitioning=Hash([l_orderkey@0], 24), input_partitions=2 │ AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_quantity], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 5 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([o_orderkey@2], 6), input_partitions=6 │ CoalesceBatchesExec: target_batch_size=8192 │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@4, o_orderdate@5] @@ -1757,17 +1757,17 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 3 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] │ RepartitionExec: partitioning=Hash([c_custkey@0], 18), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_custkey, c_name], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 4 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] │ RepartitionExec: partitioning=Hash([o_custkey@1], 18), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] └────────────────────────────────────────────────── - ┌───── Stage 6 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([l_orderkey@0], 6), input_partitions=2 │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_quantity], file_type=parquet, predicate=DynamicFilterPhysicalExpr [ true ] @@ -1779,13 +1779,13 @@ mod tests { │ SortPreservingMergeExec: [o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST], metrics=[output_rows=5, elapsed_compute=] │ [Stage 8] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 8 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 8 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=5, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ AggregateExec: mode=FinalPartitioned, gby=[c_name@0 as c_name, c_custkey@1 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@3 as o_orderdate, o_totalprice@4 as o_totalprice], aggr=[sum(lineitem.l_quantity)], metrics=[output_rows=5, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5, elapsed_compute=] │ [Stage 7] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 7 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 7 ── Tasks: t0:[p0..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=5, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=35, elapsed_compute=] @@ -1799,20 +1799,20 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=600572, elapsed_compute=] │ [Stage 6] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 2 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=5, elapsed_compute=] │ FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0], metrics=[output_rows=5, elapsed_compute=] │ AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)], metrics=[output_rows=150000, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=150000, elapsed_compute=] │ [Stage 1] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 1 ── Tasks: t0:[p0..p23] t1:[p0..p23] t2:[p0..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=150000, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] │ 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=600572, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 5 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..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=150000, elapsed_compute=] │ HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@4, o_orderdate@5], metrics=[output_rows=150000, elapsed_compute=, build_input_batches=, build_input_rows=, input_batches=, input_rows=, output_batches=, build_mem_used=, build_time=, join_time=] @@ -1821,17 +1821,17 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=150000, elapsed_compute=] │ [Stage 4] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 3 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] │ RepartitionExec: partitioning=Hash([c_custkey@0], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] │ 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=15000, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t1:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] t2:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17] + ┌───── Stage 4 ── Tasks: t0:[p0..p17] t1:[p0..p17] t2:[p0..p17] │ RepartitionExec: partitioning=Hash([o_custkey@1], 18), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] │ 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=150000, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 6 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([l_orderkey@0], 6), input_partitions=2, metrics=[fetch_time=, repartition_time=, send_time=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,p1] , metrics=[] │ 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=600572, 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=] @@ -1857,7 +1857,7 @@ mod tests { │ FilterExec: (l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2) AND (l_shipmode@5 = AIR OR l_shipmode@5 = AIR REG) AND l_shipinstruct@4 = DELIVER IN PERSON, projection=[l_partkey@0, l_quantity@1, l_extendedprice@2, l_discount@3] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], file_type=parquet, predicate=(l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2) AND (l_shipmode@5 = AIR OR l_shipmode@5 = AIR REG) AND l_shipinstruct@4 = DELIVER IN PERSON AND DynamicFilterPhysicalExpr [ l_partkey@0 >= 55 AND l_partkey@0 <= 19916 OR l_partkey@0 >= 55 AND l_partkey@0 <= 19916 OR l_partkey@0 >= 55 AND l_partkey@0 <= 19916 OR l_partkey@0 >= 55 AND l_partkey@0 <= 19916 OR l_partkey@0 >= 55 AND l_partkey@0 <= 19916 OR l_partkey@0 >= 55 AND l_partkey@0 <= 19916 ], pruning_predicate=(l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(100),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(1100),15,2 OR l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(1000),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(2000),15,2 OR l_quantity_null_count@1 != row_count@2 AND l_quantity_max@0 >= Some(2000),15,2 AND l_quantity_null_count@1 != row_count@2 AND l_quantity_min@3 <= Some(3000),15,2) AND (l_shipmode_null_count@6 != row_count@2 AND l_shipmode_min@4 <= AIR AND AIR <= l_shipmode_max@5 OR l_shipmode_null_count@6 != row_count@2 AND l_shipmode_min@4 <= AIR REG AND AIR REG <= l_shipmode_max@5) AND l_shipinstruct_null_count@9 != row_count@2 AND l_shipinstruct_min@7 <= DELIVER IN PERSON AND DELIVER IN PERSON <= l_shipinstruct_max@8 AND (l_partkey_null_count@11 != row_count@2 AND l_partkey_max@10 >= 55 AND l_partkey_null_count@11 != row_count@2 AND l_partkey_min@12 <= 19916 OR l_partkey_null_count@11 != row_count@2 AND l_partkey_max@10 >= 55 AND l_partkey_null_count@11 != row_count@2 AND l_partkey_min@12 <= 19916 OR l_partkey_null_count@11 != row_count@2 AND l_partkey_max@10 >= 55 AND l_partkey_null_count@11 != row_count@2 AND l_partkey_min@12 <= 19916 OR l_partkey_null_count@11 != row_count@2 AND l_partkey_max@10 >= 55 AND l_partkey_null_count@11 != row_count@2 AND l_partkey_min@12 <= 19916 OR l_partkey_null_count@11 != row_count@2 AND l_partkey_max@10 >= 55 AND l_partkey_null_count@11 != row_count@2 AND l_partkey_min@12 <= 19916 OR l_partkey_null_count@11 != row_count@2 AND l_partkey_max@10 >= 55 AND l_partkey_null_count@11 != row_count@2 AND l_partkey_min@12 <= 19916), required_guarantees=[l_shipinstruct in (DELIVER IN PERSON), l_shipmode in (AIR, AIR REG)] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: (p_brand@1 = Brand#12 AND p_container@3 IN ([Literal { value: Utf8View("SM CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN ([Literal { value: Utf8View("MED BAG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN ([Literal { value: Utf8View("LG CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND p_size@2 <= 15) AND p_size@2 >= 1 │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] @@ -1879,7 +1879,7 @@ mod tests { │ FilterExec: (l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2) AND (l_shipmode@5 = AIR OR l_shipmode@5 = AIR REG) AND l_shipinstruct@4 = DELIVER IN PERSON, projection=[l_partkey@0, l_quantity@1, l_extendedprice@2, l_discount@3], metrics=[output_rows=12635, elapsed_compute=] │ 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=600572, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=42, 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=42, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] @@ -1917,19 +1917,19 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = CANADA, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= CANADA AND CANADA <= n_name_max@1, required_guarantees=[n_name in (CANADA)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/part/1.parquet, /testdata/tpch/data/part/10.parquet, /testdata/tpch/data/part/11.parquet], [/testdata/tpch/data/part/12.parquet, /testdata/tpch/data/part/13.parquet, /testdata/tpch/data/part/14.parquet], [/testdata/tpch/data/part/15.parquet, /testdata/tpch/data/part/16.parquet, /testdata/tpch/data/part/2.parquet], [/testdata/tpch/data/part/3.parquet, /testdata/tpch/data/part/4.parquet, /testdata/tpch/data/part/5.parquet], [/testdata/tpch/data/part/6.parquet, /testdata/tpch/data/part/7.parquet, /testdata/tpch/data/part/8.parquet], ...]}, projection=[p_partkey, p_name], file_type=parquet, predicate=p_name@1 LIKE forest%, pruning_predicate=p_name_null_count@2 != row_count@3 AND p_name_min@0 <= foresu AND forest <= p_name_max@1, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] │ RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 6), input_partitions=2 │ AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] │ CoalesceBatchesExec: target_batch_size=8192 @@ -1964,19 +1964,19 @@ mod tests { │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=83515, elapsed_compute=] │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=3, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16, elapsed_compute=] │ FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0], metrics=[output_rows=16, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] │ 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=400, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=190, elapsed_compute=] │ FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0], metrics=[output_rows=190, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] │ 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=20000, 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 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 3 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..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=83515, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=92040, elapsed_compute=] @@ -1996,14 +1996,14 @@ mod tests { │ SortPreservingMergeExec: [numwait@1 DESC, s_name@0 ASC NULLS LAST] │ [Stage 4] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[numwait@1 DESC, s_name@0 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[s_name@0 as s_name, count(Int64(1))@1 as numwait] │ AggregateExec: mode=FinalPartitioned, gby=[s_name@0 as s_name], aggr=[count(Int64(1))] │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 3 ── Tasks: t0:[p0..p23] │ RepartitionExec: partitioning=Hash([s_name@0], 24), input_partitions=6 │ AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[count(Int64(1))] │ CoalesceBatchesExec: target_batch_size=8192 @@ -2032,13 +2032,13 @@ mod tests { │ FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/lineitem/1.parquet:.., /testdata/tpch/data/lineitem/10.parquet:.., /testdata/tpch/data/lineitem/11.parquet:..], [/testdata/tpch/data/lineitem/11.parquet:.., /testdata/tpch/data/lineitem/12.parquet:.., /testdata/tpch/data/lineitem/13.parquet:.., /testdata/tpch/data/lineitem/14.parquet:..], [/testdata/tpch/data/lineitem/14.parquet:.., /testdata/tpch/data/lineitem/15.parquet:.., /testdata/tpch/data/lineitem/16.parquet:..], [/testdata/tpch/data/lineitem/16.parquet:.., /testdata/tpch/data/lineitem/2.parquet:.., /testdata/tpch/data/lineitem/3.parquet:.., /testdata/tpch/data/lineitem/4.parquet:..], [/testdata/tpch/data/lineitem/4.parquet:.., /testdata/tpch/data/lineitem/5.parquet:.., /testdata/tpch/data/lineitem/6.parquet:.., /testdata/tpch/data/lineitem/7.parquet:..], ...]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=parquet, predicate=l_receiptdate@3 > l_commitdate@2 └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/nation/1.parquet, /testdata/tpch/data/nation/10.parquet, /testdata/tpch/data/nation/11.parquet], [/testdata/tpch/data/nation/12.parquet, /testdata/tpch/data/nation/13.parquet, /testdata/tpch/data/nation/14.parquet], [/testdata/tpch/data/nation/15.parquet, /testdata/tpch/data/nation/16.parquet, /testdata/tpch/data/nation/2.parquet], [/testdata/tpch/data/nation/3.parquet, /testdata/tpch/data/nation/4.parquet, /testdata/tpch/data/nation/5.parquet], [/testdata/tpch/data/nation/6.parquet, /testdata/tpch/data/nation/7.parquet, /testdata/tpch/data/nation/8.parquet], ...]}, projection=[n_nationkey, n_name], file_type=parquet, predicate=n_name@1 = SAUDI ARABIA, pruning_predicate=n_name_null_count@2 != row_count@3 AND n_name_min@0 <= SAUDI ARABIA AND SAUDI ARABIA <= n_name_max@1, required_guarantees=[n_name in (SAUDI ARABIA)] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] @@ -2050,14 +2050,14 @@ mod tests { │ SortPreservingMergeExec: [numwait@1 DESC, s_name@0 ASC NULLS LAST], metrics=[output_rows=47, elapsed_compute=] │ [Stage 4] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[numwait@1 DESC, s_name@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=47, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ ProjectionExec: expr=[s_name@0 as s_name, count(Int64(1))@1 as numwait], metrics=[output_rows=47, elapsed_compute=] │ AggregateExec: mode=FinalPartitioned, gby=[s_name@0 as s_name], aggr=[count(Int64(1))], metrics=[output_rows=47, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=47, elapsed_compute=] │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 3 ── Tasks: t0:[p0..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=47, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, skipped_aggregation_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=7440, elapsed_compute=] @@ -2086,13 +2086,13 @@ mod tests { │ FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1], metrics=[output_rows=379809, elapsed_compute=] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/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=600572, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=16, elapsed_compute=] │ FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0], metrics=[output_rows=16, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] │ 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=400, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=72884, elapsed_compute=] │ FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0], metrics=[output_rows=72884, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] @@ -2110,14 +2110,14 @@ mod tests { │ SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST] │ [Stage 4] => NetworkCoalesceExec: output_partitions=24, input_tasks=4 └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[cntrycode@0 ASC NULLS LAST], preserve_partitioning=[true] │ ProjectionExec: expr=[cntrycode@0 as cntrycode, count(Int64(1))@1 as numcust, sum(custsale.c_acctbal)@2 as totacctbal] │ AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)] │ CoalesceBatchesExec: target_batch_size=8192 │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1 └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 3 ── Tasks: t0:[p0..p23] │ RepartitionExec: partitioning=Hash([cntrycode@0], 24), input_partitions=6 │ AggregateExec: mode=Partial, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)] │ ProjectionExec: expr=[substr(c_phone@0, 1, 2) as cntrycode, c_acctbal@1 as c_acctbal] @@ -2131,14 +2131,14 @@ mod tests { │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4 │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/orders/1.parquet:.., /testdata/tpch/data/orders/10.parquet:.., /testdata/tpch/data/orders/11.parquet:..], [/testdata/tpch/data/orders/11.parquet:.., /testdata/tpch/data/orders/12.parquet:.., /testdata/tpch/data/orders/13.parquet:.., /testdata/tpch/data/orders/14.parquet:..], [/testdata/tpch/data/orders/14.parquet:.., /testdata/tpch/data/orders/15.parquet:.., /testdata/tpch/data/orders/16.parquet:..], [/testdata/tpch/data/orders/16.parquet:.., /testdata/tpch/data/orders/2.parquet:.., /testdata/tpch/data/orders/3.parquet:.., /testdata/tpch/data/orders/4.parquet:..], [/testdata/tpch/data/orders/4.parquet:.., /testdata/tpch/data/orders/5.parquet:.., /testdata/tpch/data/orders/6.parquet:.., /testdata/tpch/data/orders/7.parquet:..], ...]}, projection=[o_custkey], file_type=parquet └────────────────────────────────────────────────── - ┌───── Stage 1 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN ([Literal { value: Utf8View("13"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("31"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("23"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("29"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("30"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("18"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("17"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), projection=[c_acctbal@1] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] │ DataSourceExec: file_groups={6 groups: [[/testdata/tpch/data/customer/1.parquet, /testdata/tpch/data/customer/10.parquet, /testdata/tpch/data/customer/11.parquet], [/testdata/tpch/data/customer/12.parquet, /testdata/tpch/data/customer/13.parquet, /testdata/tpch/data/customer/14.parquet], [/testdata/tpch/data/customer/15.parquet, /testdata/tpch/data/customer/16.parquet, /testdata/tpch/data/customer/2.parquet], [/testdata/tpch/data/customer/3.parquet, /testdata/tpch/data/customer/4.parquet, /testdata/tpch/data/customer/5.parquet], [/testdata/tpch/data/customer/6.parquet, /testdata/tpch/data/customer/7.parquet, /testdata/tpch/data/customer/8.parquet], ...]}, projection=[c_phone, c_acctbal], file_type=parquet, predicate=c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN ([Literal { value: Utf8View("13"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("31"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("23"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("29"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("30"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("18"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("17"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]), pruning_predicate=c_acctbal_null_count@1 != row_count@2 AND c_acctbal_max@0 > Some(0),15,2, required_guarantees=[] └────────────────────────────────────────────────── - ┌───── Stage 2 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192 │ FilterExec: substr(c_phone@1, 1, 2) IN ([Literal { value: Utf8View("13"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("31"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("23"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("29"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("30"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("18"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("17"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] @@ -2151,14 +2151,14 @@ mod tests { │ SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST], metrics=[output_rows=7, elapsed_compute=] │ [Stage 4] => NetworkCoalesceExec: output_partitions=24, input_tasks=4, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 4 ── Tasks: t0:[p0,p1,p2,p3,p4,p5] t1:[p0,p1,p2,p3,p4,p5] t2:[p0,p1,p2,p3,p4,p5] t3:[p0,p1,p2,p3,p4,p5] + ┌───── Stage 4 ── Tasks: t0:[p0..p5] t1:[p0..p5] t2:[p0..p5] t3:[p0..p5] │ SortExec: expr=[cntrycode@0 ASC NULLS LAST], preserve_partitioning=[true], metrics=[output_rows=7, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, batches_split=] │ ProjectionExec: expr=[cntrycode@0 as cntrycode, count(Int64(1))@1 as numcust, sum(custsale.c_acctbal)@2 as totacctbal], metrics=[output_rows=7, elapsed_compute=] │ AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)], metrics=[output_rows=7, elapsed_compute=, spill_count=, spilled_bytes=, spilled_rows=, peak_mem_used=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=7, elapsed_compute=] │ [Stage 3] => NetworkShuffleExec: output_partitions=6, input_tasks=1, metrics=[] └────────────────────────────────────────────────── - ┌───── Stage 3 ── Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7,p8,p9,p10,p11,p12,p13,p14,p15,p16,p17,p18,p19,p20,p21,p22,p23] + ┌───── Stage 3 ── Tasks: t0:[p0..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=7, 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=641, elapsed_compute=] @@ -2172,14 +2172,14 @@ mod tests { │ [Stage 2] => NetworkCoalesceExec: output_partitions=8, input_tasks=4, metrics=[] │ 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=150000, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 1 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)], metrics=[output_rows=8, elapsed_compute=] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=3741, 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=3741, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[] │ 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=15000, 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 ── Tasks: t0:[p0,p1] t1:[p2,p3] t2:[p4,p5] t3:[p6,p7] + ┌───── Stage 2 ── Tasks: t0:[p0..p1] t1:[p2..p3] t2:[p4..p5] t3:[p6..p7] │ CoalesceBatchesExec: target_batch_size=8192, metrics=[output_rows=4115, 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=4115, elapsed_compute=] │ PartitionIsolatorExec: t0:[p0,p1,__,__,__,__] t1:[__,__,p0,p1,__,__] t2:[__,__,__,__,p0,__] t3:[__,__,__,__,__,p0] , metrics=[]